Repository: incubator-gobblin Updated Branches: refs/heads/master 462ea8cf2 -> 3be46244c
[GOBBLIN-330] make the generation of kerberos principal dynamic Closes #2182 from autumnust/keberosprincipaldynamic Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/3be46244 Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/3be46244 Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/3be46244 Branch: refs/heads/master Commit: 3be46244cb90449ea63545dec6b06e5753de7cc2 Parents: 462ea8c Author: Lei Sun <[email protected]> Authored: Mon Dec 4 15:40:15 2017 -0800 Committer: Issac Buenrostro <[email protected]> Committed: Mon Dec 4 15:40:15 2017 -0800 ---------------------------------------------------------------------- .../apache/gobblin/util/hadoop/TokenUtils.java | 28 +++++++++++++++++--- 1 file changed, 24 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/3be46244/gobblin-utility/src/main/java/org/apache/gobblin/util/hadoop/TokenUtils.java ---------------------------------------------------------------------- diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/util/hadoop/TokenUtils.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/hadoop/TokenUtils.java index 15d7cd1..5cb9723 100644 --- a/gobblin-utility/src/main/java/org/apache/gobblin/util/hadoop/TokenUtils.java +++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/hadoop/TokenUtils.java @@ -21,10 +21,13 @@ import java.io.DataOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; import java.security.PrivilegedExceptionAction; import java.util.Arrays; import java.util.List; +import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -76,6 +79,8 @@ public class TokenUtils { private static final String YARN_RESOURCEMANAGER_ADDRESS = "yarn.resourcemanager.address"; private static final String MAPRED_JOB_TRACKER = "mapred.job.tracker"; private static final String MAPREDUCE_JOBTRACKER_ADDRESS = "mapreduce.jobtracker.address"; + private static final Pattern KEYTAB_USER_PATTERN = Pattern.compile(".*\\/.*@.*"); + private static final String KERBEROS_REALM = "kerberos.realm"; /** * Get Hadoop tokens (tokens for job history server, job tracker and HDFS) using Kerberos keytab. @@ -93,10 +98,10 @@ public class TokenUtils { Configuration configuration = new Configuration(); configuration.set(HADOOP_SECURITY_AUTHENTICATION, KERBEROS); UserGroupInformation.setConfiguration(configuration); - UserGroupInformation.loginUserFromKeytab(state.getProp(KEYTAB_USER), state.getProp(KEYTAB_LOCATION)); + UserGroupInformation.loginUserFromKeytab(obtainKerberosPrincipal(state), state.getProp(KEYTAB_LOCATION)); - final Optional<String> userToProxy = Strings.isNullOrEmpty(state.getProp(USER_TO_PROXY)) - ? Optional.<String> absent() : Optional.fromNullable(state.getProp(USER_TO_PROXY)); + final Optional<String> userToProxy = Strings.isNullOrEmpty(state.getProp(USER_TO_PROXY)) ? Optional.<String>absent() + : Optional.fromNullable(state.getProp(USER_TO_PROXY)); final Configuration conf = new Configuration(); final Credentials cred = new Credentials(); @@ -111,6 +116,22 @@ public class TokenUtils { return tokenFile; } + /** + * Obtain kerberos principal in a dynamic way, where the instance's value is determined by the hostname of the machine + * that the job is currently running on. + * It will be invoked when {@link #KEYTAB_USER} is not following pattern specified in {@link #KEYTAB_USER_PATTERN}. + * @throws UnknownHostException + */ + public static String obtainKerberosPrincipal(final State state) throws UnknownHostException { + if (!state.getProp(KEYTAB_USER).matches(KEYTAB_USER_PATTERN.pattern())) { + Preconditions.checkArgument(state.contains(KERBEROS_REALM)); + return state.getProp(KEYTAB_USER) + "/" + InetAddress.getLocalHost().getCanonicalHostName() + "@" + state.getProp( + KERBEROS_REALM); + } else { + return state.getProp(KEYTAB_USER); + } + } + private static void getJhToken(Configuration conf, Credentials cred) throws IOException { YarnRPC rpc = YarnRPC.create(conf); final String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS); @@ -248,5 +269,4 @@ public class TokenUtils { return renewer; } - }
