[FLINK-8830][YARN] YarnResourceManager throws NullPointerException

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

Branch: refs/heads/release-1.5
Commit: 9442373e1b66f119e189386893426e886ad94cce
Parents: 62d9273
Author: vinoyang <[email protected]>
Authored: Sun Mar 4 17:01:35 2018 +0800
Committer: Till Rohrmann <[email protected]>
Committed: Tue Mar 20 10:19:13 2018 +0100

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


http://git-wip-us.apache.org/repos/asf/flink/blob/9442373e/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 ff2478e..79a6703 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
@@ -496,18 +496,20 @@ public final class Utils {
                        // 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);
-                       Method readTokenStorageFileMethod = 
Credentials.class.getMethod(
-                               "readTokenStorageFile", File.class, 
org.apache.hadoop.conf.Configuration.class);
-
-                       Credentials cred =
-                               (Credentials) readTokenStorageFileMethod.invoke(
-                                       null,
-                                       new File(fileLocation),
-                                       
HadoopUtils.getHadoopConfiguration(flinkConfig));
-
-                       cred.writeTokenStorageToStream(dob);
-                       ByteBuffer securityTokens = 
ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
-                       ctx.setTokens(securityTokens);
+                       if (fileLocation != null) {
+                               Method readTokenStorageFileMethod = 
Credentials.class.getMethod(
+                                       "readTokenStorageFile", File.class, 
org.apache.hadoop.conf.Configuration.class);
+
+                               Credentials cred =
+                                       (Credentials) 
readTokenStorageFileMethod.invoke(
+                                               null,
+                                               new File(fileLocation),
+                                               
HadoopUtils.getHadoopConfiguration(flinkConfig));
+
+                               cred.writeTokenStorageToStream(dob);
+                               ByteBuffer securityTokens = 
ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+                               ctx.setTokens(securityTokens);
+                       }
                }
                catch (Throwable t) {
                        log.error("Getting current user info failed when trying 
to launch the container", t);

Reply via email to