You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ki...@apache.org on 2014/04/21 20:29:35 UTC

svn commit: r1588951 - in /hadoop/common/branches/branch-2.4/hadoop-common-project: hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/ hadoop-common/ hadoop-common/src/main/java/org/apache/hadoop/http/ hadoop-common/src/main/na...

Author: kihwal
Date: Mon Apr 21 18:29:35 2014
New Revision: 1588951

URL: http://svn.apache.org/r1588951
Log:
svn merge -c 1588949 merging from trunk to branch-2.4 to fix:HADOOP-10522. JniBasedUnixGroupMapping mishandles errors. Contributed by Kihwal Lee.

Modified:
    hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
    hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
    hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c
    hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_group_info.c
    hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_user_info.c

Modified: hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java?rev=1588951&r1=1588950&r2=1588951&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java (original)
+++ hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java Mon Apr 21 18:29:35 2014
@@ -26,7 +26,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.security.auth.Subject;
-import javax.security.auth.kerberos.KerberosPrincipal;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
@@ -36,11 +35,9 @@ import javax.servlet.http.HttpServletReq
 import javax.servlet.http.HttpServletResponse;
 import java.io.File;
 import java.io.IOException;
-import java.security.Principal;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
@@ -143,7 +140,7 @@ public class KerberosAuthenticationHandl
   private String principal;
   private String keytab;
   private GSSManager gssManager;
-  private LoginContext loginContext;
+  private Subject serverSubject = new Subject();
 
   /**
    * Initializes the authentication handler instance.
@@ -176,17 +173,20 @@ public class KerberosAuthenticationHandl
         KerberosName.setRules(nameRules);
       }
       
-      Set<Principal> principals = new HashSet<Principal>();
-      principals.add(new KerberosPrincipal(principal));
-      Subject subject = new Subject(false, principals, new HashSet<Object>(), new HashSet<Object>());
-
-      KerberosConfiguration kerberosConfiguration = new KerberosConfiguration(keytab, principal);
-
       LOG.info("Login using keytab "+keytab+", for principal "+principal);
-      loginContext = new LoginContext("", subject, null, kerberosConfiguration);
-      loginContext.login();
-
-      Subject serverSubject = loginContext.getSubject();
+      for (String servicePrincipal : principal.split(",")) {
+        final KerberosConfiguration kerberosConfiguration =
+            new KerberosConfiguration(keytab, servicePrincipal);
+        final LoginContext loginContext =
+            new LoginContext("", serverSubject, null, kerberosConfiguration);
+        try {
+          loginContext.login();
+        } catch (LoginException le) {
+          LOG.warn("Failed to login as [{}]", servicePrincipal, le);
+          throw new AuthenticationException(le);          
+        }
+        serverSubject.getPrivateCredentials().add(loginContext);
+      }
       try {
         gssManager = Subject.doAs(serverSubject, new PrivilegedExceptionAction<GSSManager>() {
 
@@ -211,13 +211,17 @@ public class KerberosAuthenticationHandl
    */
   @Override
   public void destroy() {
-    try {
-      if (loginContext != null) {
-        loginContext.logout();
-        loginContext = null;
+    if (serverSubject != null) {
+      final Set<LoginContext> logins =
+          serverSubject.getPrivateCredentials(LoginContext.class);
+      for (LoginContext login : logins) {
+        try {
+          login.logout();
+        } catch (LoginException ex) {
+          LOG.warn(ex.getMessage(), ex);
+        }
       }
-    } catch (LoginException ex) {
-      LOG.warn(ex.getMessage(), ex);
+      serverSubject = null;
     }
   }
 
@@ -304,7 +308,7 @@ public class KerberosAuthenticationHandl
       authorization = authorization.substring(KerberosAuthenticator.NEGOTIATE.length()).trim();
       final Base64 base64 = new Base64(0);
       final byte[] clientToken = base64.decode(authorization);
-      Subject serverSubject = loginContext.getSubject();
+      final String serverName = request.getServerName();
       try {
         token = Subject.doAs(serverSubject, new PrivilegedExceptionAction<AuthenticationToken>() {
 
@@ -314,15 +318,15 @@ public class KerberosAuthenticationHandl
             GSSContext gssContext = null;
             GSSCredential gssCreds = null;
             try {
-              if (IBM_JAVA) {
-                // IBM JDK needs non-null credentials to be passed to createContext here, with
-                // SPNEGO mechanism specified, otherwise JGSS will use its default mechanism
-                // only, which is Kerberos V5.
-                gssCreds = gssManager.createCredential(null, GSSCredential.INDEFINITE_LIFETIME,
-                    new Oid[]{KerberosUtil.getOidInstance("GSS_SPNEGO_MECH_OID"),
-                        KerberosUtil.getOidInstance("GSS_KRB5_MECH_OID")},
-                    GSSCredential.ACCEPT_ONLY);
-              }
+              gssCreds = gssManager.createCredential(
+                  gssManager.createName(
+                      KerberosUtil.getServicePrincipal("HTTP", serverName),
+                      KerberosUtil.getOidInstance("NT_GSS_KRB5_PRINCIPAL")),
+                  GSSCredential.INDEFINITE_LIFETIME,
+                  new Oid[]{
+                    KerberosUtil.getOidInstance("GSS_SPNEGO_MECH_OID"),
+                    KerberosUtil.getOidInstance("GSS_KRB5_MECH_OID")},
+                  GSSCredential.ACCEPT_ONLY);
               gssContext = gssManager.createContext(gssCreds);
               byte[] serverToken = gssContext.acceptSecContext(clientToken, 0, clientToken.length);
               if (serverToken != null && serverToken.length > 0) {

Modified: hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1588951&r1=1588950&r2=1588951&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/CHANGES.txt Mon Apr 21 18:29:35 2014
@@ -27,6 +27,8 @@ Release 2.4.1 - UNRELEASED
     HADOOP-10490. TestMapFile and TestBloomMapFile leak file descriptors.
     (cnauroth)
 
+    HADOOP-10522. JniBasedUnixGroupMapping mishandles errors. (kihwal)
+
 Release 2.4.0 - 2014-04-07 
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java?rev=1588951&r1=1588950&r2=1588951&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java (original)
+++ hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java Mon Apr 21 18:29:35 2014
@@ -62,6 +62,7 @@ import org.apache.hadoop.security.author
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
 import org.mortbay.io.Buffer;
 import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Handler;
@@ -670,11 +671,16 @@ public class HttpServer implements Filte
   protected void initSpnego(Configuration conf,
       String usernameConfKey, String keytabConfKey) throws IOException {
     Map<String, String> params = new HashMap<String, String>();
-    String principalInConf = conf.get(usernameConfKey);
-    if (principalInConf != null && !principalInConf.isEmpty()) {
+    String[] principalsInConf = conf.getStrings(usernameConfKey);
+    if (principalsInConf != null && principalsInConf.length != 0) {
+      for (int i=0; i < principalsInConf.length; i++) {
+        principalsInConf[i] =
+            SecurityUtil.getServerPrincipal(principalsInConf[i], listener.getHost());
+      }
       params.put("kerberos.principal",
-                 SecurityUtil.getServerPrincipal(principalInConf, listener.getHost()));
+          StringUtils.join(",", principalsInConf));
     }
+
     String httpKeytab = conf.get(keytabConfKey);
     if (httpKeytab != null && !httpKeytab.isEmpty()) {
       params.put("kerberos.keytab", httpKeytab);

Modified: hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c?rev=1588951&r1=1588950&r2=1588951&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c (original)
+++ hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c Mon Apr 21 18:29:35 2014
@@ -120,10 +120,18 @@ Java_org_apache_hadoop_security_JniBased
     goto done;
   }
   ret = hadoop_user_info_fetch(uinfo, username);
-  if (ret == ENOENT) {
-    jgroups = (*env)->NewObjectArray(env, 0, g_string_clazz, NULL);
+  if (ret) {
+    if (ret == ENOENT) {
+      jgroups = (*env)->NewObjectArray(env, 0, g_string_clazz, NULL);
+    } else { // handle other errors
+      char buf[128];
+      snprintf(buf, sizeof(buf), "getgrouplist: error looking up user. %d (%s)",
+               ret, terror(ret));
+      THROW(env, "java/lang/RuntimeException", buf);
+    }
     goto done;
   }
+
   ginfo = hadoop_group_info_alloc();
   if (!ginfo) {
     THROW(env, "java/lang/OutOfMemoryError", NULL);
@@ -135,7 +143,7 @@ Java_org_apache_hadoop_security_JniBased
       THROW(env, "java/lang/OutOfMemoryError", NULL);
     } else {
       char buf[128];
-      snprintf(buf, sizeof(buf), "getgrouplist error %d (%s)",
+      snprintf(buf, sizeof(buf), "getgrouplist: error looking up groups. %d (%s)",
                ret, terror(ret));
       THROW(env, "java/lang/RuntimeException", buf);
     }

Modified: hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_group_info.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_group_info.c?rev=1588951&r1=1588950&r2=1588951&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_group_info.c (original)
+++ hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_group_info.c Mon Apr 21 18:29:35 2014
@@ -27,6 +27,8 @@
 #include <string.h>
 #include <unistd.h>
 
+#define MAX_GROUP_LOOKUP_TRIES 5
+
 struct hadoop_group_info *hadoop_group_info_alloc(void)
 {
   struct hadoop_group_info *ginfo;
@@ -84,31 +86,48 @@ static int getgrgid_error_translate(int 
 int hadoop_group_info_fetch(struct hadoop_group_info *ginfo, gid_t gid)
 {
   struct group *group;
-  int err;
+  int ret, i; 
   size_t buf_sz;
   char *nbuf;
 
   hadoop_group_info_clear(ginfo);
-  for (;;) {
-    do {
-      group = NULL;
-      err = getgrgid_r(gid, &ginfo->group, ginfo->buf,
-                         ginfo->buf_sz, &group);
-    } while ((!group) && (err == EINTR));
-    if (group) {
-      return 0;
-    }
-    if (err != ERANGE) {
-      return getgrgid_error_translate(errno);
+  for (i = 0, ret = 0; i < MAX_GROUP_LOOKUP_TRIES; i++) {
+    // If the previous call returned ERANGE, increase the buffer size
+    if (ret == ERANGE) {
+      buf_sz = ginfo->buf_sz * 2;
+      nbuf = realloc(ginfo->buf, buf_sz);
+      if (!nbuf) {
+        return ENOMEM;
+      }
+      ginfo->buf = nbuf;
+      ginfo->buf_sz = buf_sz;
     }
-    buf_sz = ginfo->buf_sz * 2;
-    nbuf = realloc(ginfo->buf, buf_sz);
-    if (!nbuf) {
-      return ENOMEM;
+
+    // The following call returns errno. Reading the global errno wihtout
+    // locking is not thread-safe.
+    group = NULL;
+    ret = getgrgid_r(gid, &ginfo->group, ginfo->buf,
+                       ginfo->buf_sz, &group);
+    switch(ret) {
+      case 0:
+        if (!group) {
+          // The underlying library likely has a bug.
+          return EIO;
+        }
+        return 0;
+      case EINTR:
+      case ERANGE:
+        // Retry on these errors.
+        // EINTR: a signal was handled and this thread was allowed to continue.
+        // ERANGE: the buffer was not big enough.
+        break;
+      default:
+        // Lookup failed.
+        return getgrgid_error_translate(ret);
     }
-    ginfo->buf = nbuf;
-    ginfo->buf_sz = buf_sz;
   }
+  // Did not succeed after the retries. Return the last error.
+  return getgrgid_error_translate(ret);
 }
 
 #ifdef GROUP_TESTING

Modified: hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_user_info.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_user_info.c?rev=1588951&r1=1588950&r2=1588951&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_user_info.c (original)
+++ hadoop/common/branches/branch-2.4/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_user_info.c Mon Apr 21 18:29:35 2014
@@ -28,6 +28,7 @@
 #include <unistd.h>
 
 #define INITIAL_GIDS_SIZE 32
+#define MAX_USER_LOOKUP_TRIES 5
 
 struct hadoop_user_info *hadoop_user_info_alloc(void)
 {
@@ -95,31 +96,48 @@ int hadoop_user_info_fetch(struct hadoop
                            const char *username)
 {
   struct passwd *pwd;
-  int err;
+  int ret, i;
   size_t buf_sz;
   char *nbuf;
 
   hadoop_user_info_clear(uinfo);
-  for (;;) {
-    do {
-      pwd = NULL;
-      err = getpwnam_r(username, &uinfo->pwd, uinfo->buf,
-                         uinfo->buf_sz, &pwd);
-    } while ((!pwd) && (errno == EINTR));
-    if (pwd) {
-      return 0;
-    }
-    if (err != ERANGE) {
-      return getpwnam_error_translate(errno);
+  for (i = 0, ret = 0; i < MAX_USER_LOOKUP_TRIES; i++) {
+    // If the previous call returned ERANGE, increase the buffer size
+    if (ret == ERANGE) {
+      buf_sz = uinfo->buf_sz * 2;
+      nbuf = realloc(uinfo->buf, buf_sz);
+      if (!nbuf) {
+        return ENOMEM;
+      }
+      uinfo->buf = nbuf;
+      uinfo->buf_sz = buf_sz;
     }
-    buf_sz = uinfo->buf_sz * 2;
-    nbuf = realloc(uinfo->buf, buf_sz);
-    if (!nbuf) {
-      return ENOMEM;
+
+    // The following call returns errno. Reading the global errno wihtout
+    // locking is not thread-safe.
+    pwd = NULL;
+    ret = getpwnam_r(username, &uinfo->pwd, uinfo->buf,
+                         uinfo->buf_sz, &pwd);
+    switch(ret) {
+      case 0:
+        if (!pwd) {
+          // The underlying library likely has a bug.
+          return EIO;
+        }
+        return 0;
+      case EINTR:
+      case ERANGE:
+        // Retry on these errors.
+        // EINTR: a signal was handled and this thread was allowed to continue.
+        // ERANGE: the buffer was not big enough.
+        break;
+      default:
+        // Lookup failed.
+        return getpwnam_error_translate(ret);
     }
-    uinfo->buf = nbuf;
-    uinfo->buf_sz = buf_sz;
   }
+  // Did not succeed after the retries. Return the last error.
+  return getpwnam_error_translate(ret);
 }
 
 static int put_primary_gid_first(struct hadoop_user_info *uinfo)