storm-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bo...@apache.org
Subject [1/3] storm git commit: STORM-2214: add in cacheing of the Login
Date Wed, 04 Jan 2017 19:14:20 GMT
Repository: storm
Updated Branches:
  refs/heads/master abee4d712 -> 6b92d236f


STORM-2214: add in cacheing of the Login


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/81b601dd
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/81b601dd
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/81b601dd

Branch: refs/heads/master
Commit: 81b601dd5589a313f26e9b5b4961eb1578cd821e
Parents: ea67694
Author: Robert (Bobby) Evans <evans@yahoo-inc.com>
Authored: Tue Nov 22 10:27:35 2016 -0600
Committer: Robert (Bobby) Evans <evans@yahoo-inc.com>
Committed: Tue Nov 22 10:27:35 2016 -0600

----------------------------------------------------------------------
 .../kerberos/KerberosSaslTransportPlugin.java   | 74 +++++++++++++++++---
 1 file changed, 63 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/81b601dd/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
index 6f1c346..81efe95 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
@@ -22,13 +22,15 @@ import java.io.IOException;
 import java.security.Principal;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
+
 import java.util.Map;
 import java.util.Set;
+import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
 import javax.security.auth.Subject;
 import javax.security.auth.callback.CallbackHandler;
 import javax.security.auth.kerberos.KerberosTicket;
-import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginException;
 import javax.security.sasl.Sasl;
@@ -50,6 +52,44 @@ import org.apache.storm.security.auth.SaslTransportPlugin;
 public class KerberosSaslTransportPlugin extends SaslTransportPlugin {
     public static final String KERBEROS = "GSSAPI"; 
     private static final Logger LOG = LoggerFactory.getLogger(KerberosSaslTransportPlugin.class);
+    private static Map <LoginCacheKey, Login> loginCache = new ConcurrentHashMap<>();
+
+    private class LoginCacheKey {
+        private String _keyString = null;
+
+        public LoginCacheKey(Configuration conf, String login_context) throws IOException
{
+            if (conf == null) {
+                throw new IllegalArgumentException("Configuration should not be null");
+            }
+            SortedMap<String, ?> configsMap = AuthUtils.pullConfig(conf, login_context);
+            if (configsMap!=null) {
+                StringBuilder stringBuilder = new StringBuilder();
+                for (String configKey: configsMap.keySet()) {
+                    String configValue = (String) configsMap.get(configKey);
+                    stringBuilder.append(configKey);
+                    stringBuilder.append(configValue);
+                }
+                _keyString = stringBuilder.toString();
+            } else {
+                throw new RuntimeException("Error in parsing the kerberos login Configuration,
returned null");
+            }
+        }
+
+        @Override
+        public int hashCode() {
+            return _keyString.hashCode();
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            return (obj instanceof LoginCacheKey) && _keyString.equals(((LoginCacheKey)obj)._keyString);
+        }
+
+        @Override
+        public String toString() {
+            return (_keyString);
+        }
+    }
 
     public TTransportFactory getServerTransportFactory() throws IOException {
         //create an authentication callback handler
@@ -59,10 +99,11 @@ public class KerberosSaslTransportPlugin extends SaslTransportPlugin {
         Subject subject = null;
         try {
             //specify a configuration object to be used
-            Configuration.setConfiguration(login_conf); 
+            Configuration.setConfiguration(login_conf);
             //now login
             Login login = new Login(AuthUtils.LOGIN_CONTEXT_SERVER, server_callback_handler);
             subject = login.getSubject();
+            login.startThreadIfNeeded();
         } catch (LoginException ex) {
             LOG.error("Server failed to login in principal:" + ex, ex);
             throw new RuntimeException(ex);
@@ -100,15 +141,26 @@ public class KerberosSaslTransportPlugin extends SaslTransportPlugin
{
         ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf);
         
         //login our user
-        Login login = null;
-        try { 
-            //specify a configuration object to be used
-            Configuration.setConfiguration(login_conf); 
-            //now login
-            login  = new Login(AuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler);
-        } catch (LoginException ex) {
-            LOG.error("Server failed to login in principal:" + ex, ex);
-            throw new RuntimeException(ex);
+        LoginCacheKey key = new LoginCacheKey(login_conf, AuthUtils.LOGIN_CONTEXT_CLIENT);
+        Login login = loginCache.get(key);
+        if (login == null) {
+            LOG.debug("Kerberos Login was not found in the Login Cache, attempting to contact
the Kerberos Server");
+            synchronized (loginCache) {
+                login = loginCache.get(key);
+                if (login == null) {
+                    try {
+                        //specify a configuration object to be used
+                        Configuration.setConfiguration(login_conf);
+                        //now login
+                        login = new Login(AuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler);
+                        login.startThreadIfNeeded();
+                        loginCache.put(key, login);
+                    } catch (LoginException ex) {
+                        LOG.error("Server failed to login in principal:" + ex, ex);
+                        throw new RuntimeException(ex);
+                    }
+                }
+            }
         }
 
         final Subject subject = login.getSubject();


Mime
View raw message