From dd3d68f63d3be46131e2dbdd05ddb6c06727e519 Mon Sep 17 00:00:00 2001 From: Prabhu Joseph Date: Wed, 1 Jul 2020 00:33:22 +0530 Subject: [PATCH] YARN-10333. YarnClient obtain Delegation Token for Log Aggregation Path --- .../yarn/client/api/impl/YarnClientImpl.java | 56 ++++++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java index 14133ba..49151c7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java @@ -30,9 +30,12 @@ import java.util.Set; import java.util.concurrent.Future; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputByteBuffer; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Text; @@ -131,6 +134,8 @@ import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileController; +import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerFactory; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -314,6 +319,16 @@ public YarnClientApplication createApplication() addTimelineDelegationToken(appContext.getAMContainerSpec()); } + // Automatically add the DT for Log Aggregation path + // This is useful when a separate storage is used for log aggregation + try { + if (isSecurityEnabled()) { + addLogAggregationDelegationToken(appContext.getAMContainerSpec()); + } + } catch (Exception e) { + LOG.warn("Failed to obtain delegation token for Log Aggregation Path", e); + } + //TODO: YARN-1763:Handle RM failovers during the submitApplication call. rmClient.submitApplication(request); @@ -373,6 +388,47 @@ public YarnClientApplication createApplication() return applicationId; } + private void addLogAggregationDelegationToken( + ContainerLaunchContext clc) throws YarnException, IOException { + Credentials credentials = new Credentials(); + DataInputByteBuffer dibb = new DataInputByteBuffer(); + ByteBuffer tokens = clc.getTokens(); + if (tokens != null) { + dibb.reset(tokens); + credentials.readTokenStorageStream(dibb); + tokens.rewind(); + } + + Configuration conf = getConfig(); + String masterPrincipal = YarnClientUtils.getRmPrincipal(conf); + if (StringUtils.isEmpty(masterPrincipal)) { + throw new IOException( + "Can't get Master Kerberos principal for use as renewer"); + } + LOG.debug("Delegation Token Renewer: " + masterPrincipal); + + LogAggregationFileControllerFactory factory = + new LogAggregationFileControllerFactory(conf); + LogAggregationFileController fileController = + factory.getFileControllerForWrite(); + Path remoteRootLogDir = fileController.getRemoteRootLogDir(); + FileSystem fs = remoteRootLogDir.getFileSystem(conf); + + final org.apache.hadoop.security.token.Token finalTokens[] = + fs.addDelegationTokens(masterPrincipal, credentials); + if (finalTokens != null) { + for (org.apache.hadoop.security.token.Token token : finalTokens) { + LOG.info("Added delegation token for log aggregation path " + + remoteRootLogDir + "; "+token); + } + } + + DataOutputBuffer dob = new DataOutputBuffer(); + credentials.writeTokenStorageToStream(dob); + tokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); + clc.setTokens(tokens); + } + private void addTimelineDelegationToken( ContainerLaunchContext clc) throws YarnException, IOException { Credentials credentials = new Credentials(); -- 2.7.4 (Apple Git-66)