[FLINK-8830] [yarn] Log reading of Hadoop's token file

This closes #5629.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4536e9cb
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4536e9cb
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4536e9cb

Branch: refs/heads/master
Commit: 4536e9cbe1925c0a6dd24f74c2b73a675afb2625
Parents: bcb0f32
Author: Till Rohrmann <[email protected]>
Authored: Mon Mar 19 18:06:12 2018 +0100
Committer: Till Rohrmann <[email protected]>
Committed: Tue Mar 20 10:18:02 2018 +0100

----------------------------------------------------------------------
 .../main/java/org/apache/flink/yarn/Utils.java  | 22 +++++++++++---------
 1 file changed, 12 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4536e9cb/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java 
b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
index 79a6703..b9f7fac 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
@@ -489,14 +489,15 @@ public final class Utils {
 
                ctx.setEnvironment(containerEnv);
 
-               try (DataOutputBuffer dob = new DataOutputBuffer()) {
-                       log.debug("Adding security tokens to Task Executor 
Container launch Context....");
+               // For TaskManager YARN container context, read the tokens from 
the jobmanager yarn container local file.
+               // NOTE: must read the tokens from the local file, not from the 
UGI context, because if UGI is login
+               // using Kerberos keytabs, there is no HDFS delegation token in 
the UGI context.
+               final String fileLocation = 
System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION);
+
+               if (fileLocation != null) {
+                       log.debug("Adding security tokens to TaskExecutor's 
container launch context.");
 
-                       // For TaskManager YARN container context, read the 
tokens from the jobmanager yarn container local flie.
-                       // NOTE: must read the tokens from the local file, not 
from the UGI context, because if UGI is login
-                       // using Kerberos keytabs, there is no HDFS delegation 
token in the UGI context.
-                       String fileLocation = 
System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION);
-                       if (fileLocation != null) {
+                       try (DataOutputBuffer dob = new DataOutputBuffer()) {
                                Method readTokenStorageFileMethod = 
Credentials.class.getMethod(
                                        "readTokenStorageFile", File.class, 
org.apache.hadoop.conf.Configuration.class);
 
@@ -509,10 +510,11 @@ public final class Utils {
                                cred.writeTokenStorageToStream(dob);
                                ByteBuffer securityTokens = 
ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
                                ctx.setTokens(securityTokens);
+                       } catch (Throwable t) {
+                               log.error("Failed to add Hadoop's security 
tokens.", t);
                        }
-               }
-               catch (Throwable t) {
-                       log.error("Getting current user info failed when trying 
to launch the container", t);
+               } else {
+                       log.info("Could not set security tokens because 
Hadoop's token file location is unknown.");
                }
 
                return ctx;

Reply via email to