This is an automated email from the ASF dual-hosted git repository.

ethanli pushed a commit to branch 2.1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/2.1.x-branch by this push:
     new fba1a5f  STORM-3606 prevent Hadoop renewal thread from running (NPEs 
and restarts worker)
fba1a5f is described below

commit fba1a5f9ead55a0e82e7c19fc10bb3bb8af5cb42
Author: Aaron Gresch <[email protected]>
AuthorDate: Wed Apr 1 16:55:31 2020 -0500

    STORM-3606 prevent Hadoop renewal thread from running (NPEs and restarts 
worker)
---
 .../storm/security/auth/kerberos/AutoTGT.java      | 45 ++++++++++++++++------
 1 file changed, 33 insertions(+), 12 deletions(-)

diff --git 
a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java 
b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
index 2e3851c..58e5a84 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
@@ -12,7 +12,10 @@
 
 package org.apache.storm.security.auth.kerberos;
 
+import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
+import java.security.Principal;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
@@ -101,10 +104,10 @@ public class AutoTGT implements IAutoCredentials, 
ICredentialsRenewer, IMetricsR
 
     public static void main(String[] args) throws Exception {
         AutoTGT at = new AutoTGT();
-        Map<String, Object> conf = new java.util.HashMap();
+        Map<String, Object> conf = new HashMap();
         conf.put("java.security.auth.login.config", args[0]);
         at.prepare(conf);
-        Map<String, String> creds = new java.util.HashMap<String, String>();
+        Map<String, String> creds = new HashMap<String, String>();
         at.populateCredentials(creds);
         Subject s = new Subject();
         at.populateSubject(s, creds);
@@ -207,19 +210,37 @@ public class AutoTGT implements IAutoCredentials, 
ICredentialsRenewer, IMetricsR
                 return;
             }
 
-            LOG.info("Invoking Hadoop 
UserGroupInformation.loginUserFromSubject.");
-            Method login = ugi.getMethod("loginUserFromSubject", 
Subject.class);
-            login.invoke(null, subject);
+            // We are just trying to do the following:
+            //
+            // Configuration conf = new Configuration();
+            // HadoopKerberosName.setConfiguration(conf);
+            // subject.getPrincipals().add(new 
User(tgt.getClient().toString(), AuthenticationMethod.KERBEROS, null));
 
-            //Refer to STORM-3606 for details
-            LOG.warn("UserGroupInformation.loginUserFromSubject will spawn a 
TGT renewal thread (\"TGT Renewer for <username>\") "
-                + "to execute \"kinit -R\" command some time before the 
current TGT expires. "
-                + "It will fail because TGT is not in the local TGT cache and 
the thread will eventually abort. "
-                + "Exceptions from this TGT renewal thread can be ignored. 
Note: TGT for the Worker is kept in memory. "
-                + "Please refer to STORM-3606 for detailed explanations");
+            Class<?> confClass = 
Class.forName("org.apache.hadoop.conf.Configuration");
+            Constructor confCons = confClass.getConstructor();
+            Object conf = confCons.newInstance();
+            Class<?> hknClass = 
Class.forName("org.apache.hadoop.security.HadoopKerberosName");
+            Method hknSetConf = hknClass.getMethod("setConfiguration", 
confClass);
+            hknSetConf.invoke(null, conf);
+
+            Class<?> authMethodClass = 
Class.forName("org.apache.hadoop.security.UserGroupInformation$AuthenticationMethod");
+            Object kerbAuthMethod = null;
+            for (Object authMethod : authMethodClass.getEnumConstants()) {
+                if ("KERBEROS".equals(authMethod.toString())) {
+                    kerbAuthMethod = authMethod;
+                    break;
+                }
+            }
+
+            Class<?> userClass = 
Class.forName("org.apache.hadoop.security.User");
+            Constructor userCons = userClass.getConstructor(String.class, 
authMethodClass, LoginContext.class);
+            userCons.setAccessible(true);
+            String name = getTGT(subject).getClient().toString();
+            Object user = userCons.newInstance(name, kerbAuthMethod, null);
+            subject.getPrincipals().add((Principal) user);
 
         } catch (Exception e) {
-            LOG.warn("Something went wrong while trying to initialize Hadoop 
through reflection. This version of hadoop "
+            LOG.error("Something went wrong while trying to initialize Hadoop 
through reflection. This version of hadoop "
                      + "may not be compatible.", e);
         }
     }

Reply via email to