Skip to content

Commit

Permalink
[FLINK-1504] support for secure HDFS access using kerberos
Browse files Browse the repository at this point in the history
This closes apache#383
  • Loading branch information
mxm authored and rmetzger committed Feb 11, 2015
1 parent aedbacf commit 82cda12
Show file tree
Hide file tree
Showing 11 changed files with 161 additions and 43 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.security.SecurityUtils;
import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
import org.apache.flink.runtime.executiongraph.ExecutionGraph;
import org.apache.flink.runtime.jobgraph.JobID;
Expand Down Expand Up @@ -807,24 +808,10 @@ public String getConfigurationDirectory() {
return configurationDirectory;
}

public static String getConfigurationDirectoryFromEnv() {
String location = null;
if (System.getenv(ENV_CONFIG_DIRECTORY) != null) {
location = System.getenv(ENV_CONFIG_DIRECTORY);
} else if (new File(CONFIG_DIRECTORY_FALLBACK_1).exists()) {
location = CONFIG_DIRECTORY_FALLBACK_1;
} else if (new File(CONFIG_DIRECTORY_FALLBACK_2).exists()) {
location = CONFIG_DIRECTORY_FALLBACK_2;
} else {
throw new RuntimeException("The configuration directory was not found. Please configure the '" +
ENV_CONFIG_DIRECTORY + "' environment variable properly.");
}
return location;
}
/**
* Reads configuration settings. The default path can be overridden
* by setting the ENV variable "FLINK_CONF_DIR".
*
*
* @return Flink's global configuration
*/
protected Configuration getGlobalConfiguration() {
Expand Down Expand Up @@ -857,11 +844,25 @@ protected Configuration getGlobalConfiguration() {
System.err.println("Error while loading YARN properties: "+e.getMessage());
e.printStackTrace();
}

globalConfigurationLoaded = true;
}
return GlobalConfiguration.getConfiguration();
}
public static String getConfigurationDirectoryFromEnv() {
String location = null;
if (System.getenv(ENV_CONFIG_DIRECTORY) != null) {
location = System.getenv(ENV_CONFIG_DIRECTORY);
} else if (new File(CONFIG_DIRECTORY_FALLBACK_1).exists()) {
location = CONFIG_DIRECTORY_FALLBACK_1;
} else if (new File(CONFIG_DIRECTORY_FALLBACK_2).exists()) {
location = CONFIG_DIRECTORY_FALLBACK_2;
} else {
throw new RuntimeException("The configuration directory was not found. Please configure the '" +
ENV_CONFIG_DIRECTORY + "' environment variable properly.");
}
return location;
}

protected FiniteDuration getAkkaTimeout(){
Configuration config = getGlobalConfiguration();
Expand Down Expand Up @@ -1057,10 +1058,24 @@ public int parseParameters(String[] args) {
String action = args[0];

// remove action from parameters
String[] params = Arrays.copyOfRange(args, 1, args.length);
final String[] params = Arrays.copyOfRange(args, 1, args.length);

// do action
if (action.equals(ACTION_RUN)) {
// run() needs to run in a secured environment for the optimizer.
if(SecurityUtils.isSecurityEnabled()) {
System.out.println("Secure Hadoop setup detected.");
try {
return SecurityUtils.runSecured(new SecurityUtils.FlinkSecuredRunner<Integer>() {
@Override
public Integer run() throws Exception {
return CliFrontend.this.run(params);
}
});
} catch (Exception e) {
handleError(e);
}
}
return run(params);
} else if (action.equals(ACTION_LIST)) {
return list(params);
Expand Down
2 changes: 2 additions & 0 deletions flink-dist/src/main/flink-bin/bin/config.sh
Original file line number Diff line number Diff line change
Expand Up @@ -205,6 +205,8 @@ if [ -n "$HADOOP_HOME" ]; then
fi
fi

INTERNAL_HADOOP_CLASSPATHS="$HADOOP_CLASSPATH:$HADOOP_CONF_DIR:$YARN_CONF_DIR"

# Auxilliary function which extracts the name of host from a line which
# also potentialy includes topology information and the taskManager type
extractHostName() {
Expand Down
2 changes: 1 addition & 1 deletion flink-dist/src/main/flink-bin/bin/flink
Original file line number Diff line number Diff line change
Expand Up @@ -51,4 +51,4 @@ log_setting="-Dlog.file="$log" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4
export FLINK_CONF_DIR

# Add HADOOP_CLASSPATH to allow the usage of Hadoop file systems
$JAVA_RUN $JVM_ARGS $log_setting -classpath $CC_CLASSPATH:$HADOOP_CLASSPATH:$HADOOP_CONF_DIR:$YARN_CONF_DIR org.apache.flink.client.CliFrontend $*
$JAVA_RUN $JVM_ARGS $log_setting -classpath $CC_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS org.apache.flink.client.CliFrontend $*
3 changes: 2 additions & 1 deletion flink-dist/src/main/flink-bin/bin/jobmanager.sh
Original file line number Diff line number Diff line change
Expand Up @@ -95,8 +95,9 @@ case $STARTSTOP in
rotateLogFile $out

echo Starting job manager
$JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} "${log_setting[@]}" -classpath "$FLINK_JM_CLASSPATH:$HADOOP_CLASSPATH" org.apache.flink.runtime.jobmanager.JobManager --executionMode $EXECUTIONMODE --configDir "$FLINK_CONF_DIR" > "$out" 2>&1 < /dev/null &
$JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} "${log_setting[@]}" -classpath "$FLINK_JM_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS" org.apache.flink.runtime.jobmanager.JobManager --executionMode $EXECUTIONMODE --configDir "$FLINK_CONF_DIR" > "$out" 2>&1 < /dev/null &
echo $! > $pid

;;

(stop)
Expand Down
2 changes: 1 addition & 1 deletion flink-dist/src/main/flink-bin/bin/start-cluster.sh
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,6 @@ do
read line || GOON=false
if [ -n "$line" ]; then
HOST=$( extractHostName $line)
ssh -n $FLINK_SSH_OPTS $HOST -- "nohup /bin/bash $FLINK_BIN_DIR/taskmanager.sh start &"
ssh -n $FLINK_SSH_OPTS $HOST -- "nohup /bin/bash -l $FLINK_BIN_DIR/taskmanager.sh start &"
fi
done < "$HOSTLIST"
3 changes: 2 additions & 1 deletion flink-dist/src/main/flink-bin/bin/taskmanager.sh
Original file line number Diff line number Diff line change
Expand Up @@ -83,8 +83,9 @@ case $STARTSTOP in
rotateLogFile $out

echo Starting task manager on host $HOSTNAME
$JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} "${log_setting[@]}" -classpath "$FLINK_TM_CLASSPATH:$HADOOP_CLASSPATH" org.apache.flink.runtime.taskmanager.TaskManager --configDir "$FLINK_CONF_DIR" > "$out" 2>&1 < /dev/null &
$JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} "${log_setting[@]}" -classpath "$FLINK_TM_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS" org.apache.flink.runtime.taskmanager.TaskManager --configDir "$FLINK_CONF_DIR" > "$out" 2>&1 < /dev/null &
echo $! > $pid

;;

(stop)
Expand Down
2 changes: 1 addition & 1 deletion flink-dist/src/main/flink-bin/bin/webclient.sh
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ case $STARTSTOP in
fi
fi
echo Starting Flink webclient
$JAVA_RUN $JVM_ARGS "${log_setting[@]}" -classpath "$FLINK_WEBCLIENT_CLASSPATH:$HADOOP_CLASSPATH" org.apache.flink.client.WebFrontend --configDir "$FLINK_CONF_DIR" > "$out" 2>&1 < /dev/null &
$JAVA_RUN $JVM_ARGS "${log_setting[@]}" -classpath "$FLINK_WEBCLIENT_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS" org.apache.flink.client.WebFrontend --configDir "$FLINK_CONF_DIR" > "$out" 2>&1 < /dev/null &
echo $! > $pid
;;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.security;

import org.apache.flink.runtime.operators.MapPartitionDriver;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.security.PrivilegedExceptionAction;

public class SecurityUtils {

private static final Logger LOG = LoggerFactory.getLogger(MapPartitionDriver.class);

// load Hadoop configuration when loading the security utils.
private static Configuration hdConf = new Configuration();


public static boolean isSecurityEnabled() {
UserGroupInformation.setConfiguration(hdConf);
return UserGroupInformation.isSecurityEnabled();
}

public static <T> T runSecured(final FlinkSecuredRunner<T> runner) throws Exception {
UserGroupInformation.setConfiguration(hdConf);
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
if (!ugi.hasKerberosCredentials()) {
LOG.error("Security is enabled but no Kerberos credentials have been found. " +
"You may authenticate using the kinit command.");
}
T ret = ugi.doAs(new PrivilegedExceptionAction<T>() {
@Override
public T run() throws Exception {
return runner.run();
}
});
return ret;
}

public static interface FlinkSecuredRunner<T> {
public T run() throws Exception;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,15 @@ import java.net.InetSocketAddress
import akka.actor.Status.Failure
import akka.actor._
import akka.pattern.ask
import org.apache.flink.configuration.{ConfigConstants, GlobalConfiguration, Configuration}
import org.apache.flink.configuration.{Configuration, ConfigConstants, GlobalConfiguration}
import org.apache.flink.core.io.InputSplitAssigner
import org.apache.flink.runtime.blob.BlobServer
import org.apache.flink.runtime.executiongraph.{ExecutionJobVertex, ExecutionGraph}
import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph
import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
import org.apache.flink.runtime.messages.Messages.Acknowledge
import org.apache.flink.runtime.security.SecurityUtils
import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner
import org.apache.flink.runtime.taskmanager.TaskManager
import org.apache.flink.runtime.util.EnvironmentInformation
import org.apache.flink.runtime.{JobException, ActorLogMessages}
Expand Down Expand Up @@ -551,6 +553,20 @@ object JobManager {
EnvironmentInformation.logEnvironmentInfo(LOG, "JobManager")
val (configuration, executionMode, listeningAddress) = parseArgs(args)

if(SecurityUtils.isSecurityEnabled) {
LOG.info("Security is enabled. Starting secure JobManager.")
SecurityUtils.runSecured(new FlinkSecuredRunner[Unit] {
override def run(): Unit = {
start(configuration, executionMode, listeningAddress)
}
})
} else {
start(configuration, executionMode, listeningAddress)
}
}

def start(configuration: Configuration, executionMode: ExecutionMode,
listeningAddress : Option[(String, Int)]): Unit = {
val jobManagerSystem = AkkaUtils.createActorSystem(configuration, listeningAddress)

startActor(Props(new JobManager(configuration) with WithWebServer))(jobManagerSystem)
Expand All @@ -573,10 +589,10 @@ object JobManager {
def parseArgs(args: Array[String]): (Configuration, ExecutionMode, Option[(String, Int)]) = {
val parser = new scopt.OptionParser[JobManagerCLIConfiguration]("jobmanager") {
head("flink jobmanager")
opt[String]("configDir") action { (x, c) => c.copy(configDir = x) } text ("Specify " +
opt[String]("configDir") action { (arg, c) => c.copy(configDir = arg) } text ("Specify " +
"configuration directory.")
opt[String]("executionMode") optional() action { (x, c) =>
if(x.equals("local")){
opt[String]("executionMode") optional() action { (arg, c) =>
if(arg.equals("local")){
c.copy(executionMode = LOCAL)
}else{
c.copy(executionMode = CLUSTER)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,8 @@ import org.apache.flink.runtime.messages.TaskManagerProfilerMessages
.{UnregisterProfilingListener, UnmonitorTask, MonitorTask, RegisterProfilingListener}
import org.apache.flink.runtime.net.NetUtils
import org.apache.flink.runtime.profiling.ProfilingUtils
import org.apache.flink.runtime.security.SecurityUtils
import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner
import org.apache.flink.runtime.util.EnvironmentInformation
import org.apache.flink.util.ExceptionUtils
import org.slf4j.LoggerFactory
Expand Down Expand Up @@ -668,9 +670,22 @@ object TaskManager {

def main(args: Array[String]): Unit = {
EnvironmentInformation.logEnvironmentInfo(LOG, "TaskManager")

val (hostname, port, configuration) = parseArgs(args)

if(SecurityUtils.isSecurityEnabled) {
LOG.info("Security is enabled. Starting secure TaskManager.")
SecurityUtils.runSecured(new FlinkSecuredRunner[Unit] {
override def run(): Unit = {
startActor(hostname, port, configuration)
}
})
} else {
startActor(hostname, port, configuration)
}
}

def startActor(hostname: String, port: Int, configuration: Configuration) : Unit = {

val (taskManagerSystem, _) = startActorSystemAndActor(hostname, port, configuration,
localAkkaCommunication = false, localTaskManagerCommunication = false)

Expand Down
39 changes: 23 additions & 16 deletions flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -325,25 +325,32 @@ protected AbstractFlinkYarnCluster deployInternal(String clusterName) throws Exc

// ------------------ Check if the specified queue exists --------------

List<QueueInfo> queues = yarnClient.getAllQueues();
if(queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session.
boolean queueFound = false;
for (QueueInfo queue : queues) {
if (queue.getQueueName().equals(this.yarnQueue)) {
queueFound = true;
break;
}
}
if (!queueFound) {
String queueNames = "";
try {
List<QueueInfo> queues = yarnClient.getAllQueues();
if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session.
boolean queueFound = false;
for (QueueInfo queue : queues) {
queueNames += queue.getQueueName() + ", ";
if (queue.getQueueName().equals(this.yarnQueue)) {
queueFound = true;
break;
}
}
throw new YarnDeploymentException("The specified queue '" + this.yarnQueue + "' does not exist. " +
"Available queues: " + queueNames);
if (!queueFound) {
String queueNames = "";
for (QueueInfo queue : queues) {
queueNames += queue.getQueueName() + ", ";
}
LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " +
"Available queues: " + queueNames);
}
} else {
LOG.debug("The YARN cluster does not have any queues configured");
}
} catch(Throwable e) {
LOG.warn("Error while getting queue information from YARN: "+e.getMessage());
if(LOG.isDebugEnabled()) {
LOG.debug("Error details", e);
}
} else {
LOG.debug("The YARN cluster does not have any queues configured");
}

// ------------------ Check if the YARN Cluster has the requested resources --------------
Expand Down

0 comments on commit 82cda12

Please sign in to comment.