storm-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bo...@apache.org
Subject [1/4] storm git commit: Fixing Kerberos TGT failure issues caused by sharing single instance by multiple subjects
Date Thu, 03 Mar 2016 20:24:42 GMT
Repository: storm
Updated Branches:
  refs/heads/master df54280e3 -> 672c8951b


Fixing Kerberos TGT failure issues caused by  sharing single instance by multiple subjects

Conflicts:
	storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java
	storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java
	storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java


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

Branch: refs/heads/master
Commit: 2ab6a84eb8715d2e6a4514bf079b8d893f6d15f7
Parents: df54280
Author: Kishor Patil <kpatil@yahoo-inc.com>
Authored: Fri Feb 12 16:39:57 2016 -0600
Committer: Kishor Patil <kpatil@yahoo-inc.com>
Committed: Wed Mar 2 23:26:07 2016 +0000

----------------------------------------------------------------------
 .../apache/storm/security/auth/AuthUtils.java   | 40 ++++++++++++
 .../storm/security/auth/kerberos/AutoTGT.java   | 64 +++++++++-----------
 .../auth/kerberos/AutoTGTKrb5LoginModule.java   |  8 ++-
 3 files changed, 74 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/2ab6a84e/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java b/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java
index 86e1148..72b7d7c 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java
@@ -17,10 +17,16 @@
  */
 package org.apache.storm.security.auth;
 
+import javax.security.auth.kerberos.KerberosTicket;
 import org.apache.storm.Config;
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.Subject;
+import javax.xml.bind.DatatypeConverter;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.security.URIParameter;
 import java.security.MessageDigest;
 
@@ -345,4 +351,38 @@ public class AuthUtils {
             throw new RuntimeException(e);
         }
     }
+
+    public static byte[] serializeKerberosTicket(KerberosTicket tgt) throws Exception {
+        ByteArrayOutputStream bao = new ByteArrayOutputStream();
+        ObjectOutputStream out = new ObjectOutputStream(bao);
+        out.writeObject(tgt);
+        out.flush();
+        out.close();
+        return bao.toByteArray();
+    }
+
+    public static KerberosTicket deserializeKerberosTicket(byte[] tgtBytes) {
+        KerberosTicket ret;
+        try {
+
+            ByteArrayInputStream bin = new ByteArrayInputStream(tgtBytes);
+            ObjectInputStream in = new ObjectInputStream(bin);
+            ret = (KerberosTicket)in.readObject();
+            in.close();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+        return ret;
+    }
+
+    public static KerberosTicket cloneKerberosTicket(KerberosTicket kerberosTicket) {
+        if(kerberosTicket != null) {
+            try {
+                return (deserializeKerberosTicket(serializeKerberosTicket(kerberosTicket)));
+            } catch (Exception e) {
+                throw new RuntimeException("Failed to clone KerberosTicket TGT!!", e);
+            }
+        }
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/2ab6a84e/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
index 2590ce4..c3f8560 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
@@ -24,10 +24,6 @@ import org.apache.storm.security.auth.AuthUtils;
 
 import java.util.Map;
 import java.util.Set;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
 import java.lang.reflect.Method;
 import java.lang.reflect.Constructor;
 import java.security.Principal;
@@ -110,12 +106,9 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer
{
 
     public static void saveTGT(KerberosTicket tgt, Map<String, String> credentials)
{
         try {
-            ByteArrayOutputStream bao = new ByteArrayOutputStream();
-            ObjectOutputStream out = new ObjectOutputStream(bao);
-            out.writeObject(tgt);
-            out.flush();
-            out.close();
-            credentials.put("TGT", DatatypeConverter.printBase64Binary(bao.toByteArray()));
+
+            byte[] bytes = AuthUtils.serializeKerberosTicket(tgt);
+            credentials.put("TGT", DatatypeConverter.printBase64Binary(bytes));
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
@@ -123,15 +116,8 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer
{
 
     public static KerberosTicket getTGT(Map<String, String> credentials) {
         KerberosTicket ret = null;
-        if (credentials != null && credentials.containsKey("TGT")) {
-            try {
-                ByteArrayInputStream bin = new ByteArrayInputStream(DatatypeConverter.parseBase64Binary(credentials.get("TGT")));
-                ObjectInputStream in = new ObjectInputStream(bin);
-                ret = (KerberosTicket)in.readObject();
-                in.close();
-            } catch (Exception e) {
-                throw new RuntimeException(e);
-            }
+        if (credentials != null && credentials.containsKey("TGT") && credentials.get("TGT")
!= null) {
+            ret = AuthUtils.deserializeKerberosTicket(DatatypeConverter.parseBase64Binary(credentials.get("TGT")));
         }
         return ret;
     }
@@ -150,23 +136,7 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer
{
     private void populateSubjectWithTGT(Subject subject, Map<String, String> credentials)
{
         KerberosTicket tgt = getTGT(credentials);
         if (tgt != null) {
-            Set<Object> creds = subject.getPrivateCredentials();
-            synchronized(creds) {
-                Iterator<Object> iterator = creds.iterator();
-                while (iterator.hasNext()) {
-                    Object o = iterator.next();
-                    if (o instanceof KerberosTicket) {
-                        KerberosTicket t = (KerberosTicket)o;
-                        iterator.remove();
-                        try {
-                            t.destroy();
-                        } catch (DestroyFailedException  e) {
-                            LOG.warn("Failed to destroy ticket ", e);
-                        }
-                    }
-                }
-                creds.add(tgt);
-            }
+            clearCredentials(subject, tgt);
             subject.getPrincipals().add(tgt.getClient());
             kerbTicket.set(tgt);
         } else {
@@ -174,6 +144,28 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer
{
         }
     }
 
+    public static void clearCredentials(Subject subject, KerberosTicket tgt) {
+        Set<Object> creds = subject.getPrivateCredentials();
+        synchronized(creds) {
+            Iterator<Object> iterator = creds.iterator();
+            while (iterator.hasNext()) {
+                Object o = iterator.next();
+                if (o instanceof KerberosTicket) {
+                    KerberosTicket t = (KerberosTicket)o;
+                    iterator.remove();
+                    try {
+                        t.destroy();
+                    } catch (DestroyFailedException e) {
+                        LOG.warn("Failed to destory ticket ", e);
+                    }
+                }
+            }
+            if(tgt != null) {
+                creds.add(tgt);
+            }
+        }
+    }
+
     /**
      * Hadoop does not just go off of a TGT, it needs a bit more.  This
      * should fill in the rest.

http://git-wip-us.apache.org/repos/asf/storm/blob/2ab6a84e/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
index fd01297..c2b37e3 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
@@ -21,6 +21,7 @@ package org.apache.storm.security.auth.kerberos;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.storm.security.auth.AuthUtils;
 import java.security.Principal;
 import java.util.Map;
 import javax.security.auth.Subject;
@@ -79,7 +80,10 @@ public class AutoTGTKrb5LoginModule implements LoginModule {
             throw new LoginException("Authentication failed because the Subject is invalid.");
         }
         // Let us add the kerbClientPrinc and kerbTicket
-        subject.getPrivateCredentials().add(kerbTicket);
+        // We need to clone the ticket because java.security.auth.kerberos assumes TGT is
unique for each subject
+        // So, sharing TGT with multiple subjects can cause expired TGT to never refresh.
+        KerberosTicket kerbTicketCopy = AuthUtils.cloneKerberosTicket(kerbTicket);
+        subject.getPrivateCredentials().add(kerbTicketCopy);
         subject.getPrincipals().add(getKerbTicketClient());
         LOG.debug("Commit Succeeded.");
         return true;
@@ -96,7 +100,7 @@ public class AutoTGTKrb5LoginModule implements LoginModule {
     public boolean logout() throws LoginException {
         if (subject != null && !subject.isReadOnly() && kerbTicket != null)
{
             subject.getPrincipals().remove(kerbTicket.getClient());
-            subject.getPrivateCredentials().remove(kerbTicket);
+            AutoTGT.clearCredentials(subject, null);
         }
         kerbTicket = null;
         return true;


Mime
View raw message