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 om...@apache.org on 2011/03/04 06:37:53 UTC

svn commit: r1077816 - in /hadoop/common/branches/branch-0.20-security-202/src: core/org/apache/hadoop/ipc/ core/org/apache/hadoop/security/ core/org/apache/hadoop/security/authorize/ test/org/apache/hadoop/security/

Author: omalley
Date: Fri Mar  4 05:37:53 2011
New Revision: 1077816

URL: http://svn.apache.org/viewvc?rev=1077816&view=rev
Log:
commit 9e998f9003fc036618a5e816b857cff0efb76771
Author: Kan Zhang <ka...@yahoo-inc.com>
Date:   Thu Jan 20 13:40:36 2011 -0800

    Revert "patch based on 202"
    
    This reverts commit 9d0aeda550c5ba191e3994c91c7da54e90eefc9f.

Modified:
    hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Client.java
    hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Server.java
    hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/SecurityUtil.java
    hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
    hadoop/common/branches/branch-0.20-security-202/src/test/org/apache/hadoop/security/TestSecurityUtil.java

Modified: hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Client.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Client.java?rev=1077816&r1=1077815&r2=1077816&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Client.java (original)
+++ hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Client.java Fri Mar  4 05:37:53 2011
@@ -1230,7 +1230,7 @@ public class Client {
                    + protocol.getCanonicalName());
          }
          return SecurityUtil.getServerPrincipal(conf.get(serverKey), address
-             .getAddress());
+             .getAddress().getCanonicalHostName());
        }
        return null;
      }

Modified: hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Server.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Server.java?rev=1077816&r1=1077815&r2=1077816&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Server.java (original)
+++ hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/ipc/Server.java Fri Mar  4 05:37:53 2011
@@ -838,8 +838,8 @@ public abstract class Server {
     // Cache the remote host & port info so that even if the socket is 
     // disconnected, we can say where it used to connect to.
     private String hostAddress;
+    private String hostName;
     private int remotePort;
-    private InetAddress addr;
     
     ConnectionHeader header = new ConnectionHeader();
     Class<?> protocol;
@@ -876,11 +876,12 @@ public abstract class Server {
       this.unwrappedData = null;
       this.unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
       this.socket = channel.socket();
-      this.addr = socket.getInetAddress();
+      InetAddress addr = socket.getInetAddress();
       if (addr == null) {
         this.hostAddress = "*Unknown*";
       } else {
         this.hostAddress = addr.getHostAddress();
+        this.hostName = addr.getCanonicalHostName();
       }
       this.remotePort = socket.getPort();
       this.responseQueue = new LinkedList<Call>();
@@ -903,8 +904,8 @@ public abstract class Server {
       return hostAddress;
     }
 
-    public InetAddress getHostInetAddress() {
-      return addr;
+    public String getHostName() {
+      return hostName;
     }
     
     public void setLastContact(long lastContact) {
@@ -1312,7 +1313,7 @@ public abstract class Server {
             && (authMethod != AuthMethod.DIGEST)) {
           ProxyUsers.authorize(user, this.getHostAddress(), conf);
         }
-        authorize(user, header, getHostInetAddress());
+        authorize(user, header, getHostName());
         if (LOG.isDebugEnabled()) {
           LOG.debug("Successfully authorized " + header);
         }
@@ -1636,12 +1637,12 @@ public abstract class Server {
    * 
    * @param user client user
    * @param connection incoming connection
-   * @param addr InetAddress of incoming connection
+   * @param hostname fully-qualified domain name of incoming connection
    * @throws AuthorizationException when the client isn't authorized to talk the protocol
    */
   public void authorize(UserGroupInformation user, 
                         ConnectionHeader connection,
-                        InetAddress addr
+                        String hostname
                         ) throws AuthorizationException {
     if (authorize) {
       Class<?> protocol = null;
@@ -1651,7 +1652,7 @@ public abstract class Server {
         throw new AuthorizationException("Unknown protocol: " + 
                                          connection.getProtocol());
       }
-      ServiceAuthorizationManager.authorize(user, protocol, getConf(), addr);
+      ServiceAuthorizationManager.authorize(user, protocol, getConf(), hostname);
     }
   }
   

Modified: hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/SecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/SecurityUtil.java?rev=1077816&r1=1077815&r2=1077816&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/SecurityUtil.java (original)
+++ hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/SecurityUtil.java Fri Mar  4 05:37:53 2011
@@ -114,8 +114,8 @@ public class SecurityUtil {
   }
   
   /**
-   * Convert Kerberos principal name pattern to valid Kerberos principal
-   * names. It replaces hostname pattern with hostname, which should be
+   * Convert Kerberos principal name conf values to valid Kerberos principal
+   * names. It replaces $host in the conf values with hostname, which should be
    * fully-qualified domain name. If hostname is null or "0.0.0.0", it uses
    * dynamically looked-up fqdn of the current host instead.
    * 
@@ -128,57 +128,24 @@ public class SecurityUtil {
    */
   public static String getServerPrincipal(String principalConfig,
       String hostname) throws IOException {
-    String[] components = getComponents(principalConfig);
-    if (components == null || components.length != 3
-        || !components[1].equals(HOSTNAME_PATTERN)) {
-      return principalConfig;
-    } else {
-      return replacePattern(components, hostname);
-    }
-  }
-  
-  /**
-   * Convert Kerberos principal name pattern to valid Kerberos principal names.
-   * This method is similar to {@link #getServerPrincipal(String, String)},
-   * except 1) the reverse DNS lookup from addr to hostname is done only when
-   * necessary, 2) param addr can't be null (no default behavior of using local
-   * hostname when addr is null).
-   * 
-   * @param principalConfig
-   *          Kerberos principal name pattern to convert
-   * @param addr
-   *          InetAddress of the host used for substitution
-   * @return converted Kerberos principal name
-   * @throws IOException
-   */
-  public static String getServerPrincipal(String principalConfig,
-      InetAddress addr) throws IOException {
-    String[] components = getComponents(principalConfig);
-    if (components == null || components.length != 3
-        || !components[1].equals(HOSTNAME_PATTERN)) {
-      return principalConfig;
-    } else {
-      if (addr == null) {
-        throw new IOException("Can't replace " + HOSTNAME_PATTERN
-            + " pattern since client address is null");
-      }
-      return replacePattern(components, addr.getCanonicalHostName());
-    }
-  }
-  
-  private static String[] getComponents(String principalConfig) {
     if (principalConfig == null)
       return null;
-    return principalConfig.split("[/@]");
-  }
-  
-  private static String replacePattern(String[] components, String hostname)
-      throws IOException {
-    String fqdn = hostname;
-    if (fqdn == null || fqdn.equals("") || fqdn.equals("0.0.0.0")) {
-      fqdn = getLocalHostName();
+    String[] components = principalConfig.split("[/@]");
+    if (components.length != 3) {
+      throw new IOException(
+          "Kerberos service principal name isn't configured properly "
+              + "(should have 3 parts): " + principalConfig);
+    }
+
+    if (components[1].equals(HOSTNAME_PATTERN)) {
+      String fqdn = hostname;
+      if (fqdn == null || fqdn.equals("") || fqdn.equals("0.0.0.0")) {
+        fqdn = getLocalHostName();
+      }
+      return components[0] + "/" + fqdn + "@" + components[2];
+    } else {
+      return principalConfig;
     }
-    return components[0] + "/" + fqdn + "@" + components[2];
   }
   
   static String getLocalHostName() throws UnknownHostException {

Modified: hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java?rev=1077816&r1=1077815&r2=1077816&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java (original)
+++ hadoop/common/branches/branch-0.20-security-202/src/core/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java Fri Mar  4 05:37:53 2011
@@ -18,7 +18,6 @@
 package org.apache.hadoop.security.authorize;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.IdentityHashMap;
 import java.util.Map;
 
@@ -27,6 +26,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.KerberosName;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -65,13 +65,13 @@ public class ServiceAuthorizationManager
    * @param user user accessing the service 
    * @param protocol service being accessed
    * @param conf configuration to use
-   * @param addr InetAddress of the client
+   * @param hostname fully qualified domain name of the client
    * @throws AuthorizationException on authorization failure
    */
   public static void authorize(UserGroupInformation user, 
                                Class<?> protocol,
                                Configuration conf,
-                               InetAddress addr
+                               String hostname
                                ) throws AuthorizationException {
     AccessControlList acl = protocolToAcl.get(protocol);
     if (acl == null) {
@@ -85,24 +85,39 @@ public class ServiceAuthorizationManager
     if (krbInfo != null) {
       String clientKey = krbInfo.clientPrincipal();
       if (clientKey != null && !clientKey.equals("")) {
+        if (hostname == null) {
+          throw new AuthorizationException(
+              "Can't authorize client when client hostname is null");
+        }
         try {
           clientPrincipal = SecurityUtil.getServerPrincipal(
-              conf.get(clientKey), addr);
+              conf.get(clientKey), hostname);
         } catch (IOException e) {
           throw (AuthorizationException) new AuthorizationException(
               "Can't figure out Kerberos principal name for connection from "
-                  + addr + " for user=" + user + " protocol=" + protocol)
+                  + hostname + " for user=" + user + " protocol=" + protocol)
               .initCause(e);
         }
       }
     }
-    if((clientPrincipal != null && !clientPrincipal.equals(user.getUserName())) || 
+    // when authorizing use the short name only
+    String shortName = clientPrincipal;
+    if(clientPrincipal != null ) {
+      try {
+        shortName = new KerberosName(clientPrincipal).getShortName();
+      } catch (IOException e) {
+        LOG.warn("couldn't get short name from " + clientPrincipal, e);
+        // just keep going
+      }
+    }
+    LOG.debug("for protocol authorization compare (" + clientPrincipal + "): " 
+        + shortName + " with " + user.getShortUserName());
+    if((shortName != null &&  !shortName.equals(user.getShortUserName())) || 
         !acl.isUserAllowed(user)) {
-      AUDITLOG.warn(AUTHZ_FAILED_FOR + user + " for protocol=" + protocol
-          + ", expected client Kerberos principal is " + clientPrincipal);
+      AUDITLOG.warn(AUTHZ_FAILED_FOR + user + " for protocol="+protocol);
       throw new AuthorizationException("User " + user + 
-          " is not authorized for protocol " + protocol + 
-          ", expected client Kerberos principal is " + clientPrincipal);
+          " is not authorized for protocol " + 
+          protocol);
     }
     AUDITLOG.info(AUTHZ_SUCCESSFULL_FOR + user + " for protocol="+protocol);
   }

Modified: hadoop/common/branches/branch-0.20-security-202/src/test/org/apache/hadoop/security/TestSecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-202/src/test/org/apache/hadoop/security/TestSecurityUtil.java?rev=1077816&r1=1077815&r2=1077816&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-202/src/test/org/apache/hadoop/security/TestSecurityUtil.java (original)
+++ hadoop/common/branches/branch-0.20-security-202/src/test/org/apache/hadoop/security/TestSecurityUtil.java Fri Mar  4 05:37:53 2011
@@ -19,10 +19,8 @@ package org.apache.hadoop.security;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
-import java.net.InetAddress;
 
 import org.junit.Test;
-import org.mockito.Mockito;
 
 public class TestSecurityUtil {
   @Test
@@ -40,46 +38,27 @@ public class TestSecurityUtil {
   
   private void verify(String original, String hostname, String expected)
       throws IOException {
-    assertEquals(expected, 
-                 SecurityUtil.getServerPrincipal(original, hostname));
-
-    InetAddress addr = mockAddr(hostname);
-    assertEquals(expected, 
-                 SecurityUtil.getServerPrincipal(original, addr));
+    assertTrue(SecurityUtil.getServerPrincipal(original, hostname).equals(
+        expected));
+    assertTrue(SecurityUtil.getServerPrincipal(original, null).equals(
+        expected));
+    assertTrue(SecurityUtil.getServerPrincipal(original, "").equals(
+        expected));
+    assertTrue(SecurityUtil.getServerPrincipal(original, "0.0.0.0").equals(
+        expected));
   }
 
-  private InetAddress mockAddr(String reverseTo) {
-    InetAddress mock = Mockito.mock(InetAddress.class);
-    Mockito.doReturn(reverseTo).when(mock).getCanonicalHostName();
-    return mock;
-  }
-  
   @Test
   public void testGetServerPrincipal() throws IOException {
     String service = "hdfs/";
     String realm = "@REALM";
-    String hostname = "foohost";
-    String userPrincipal = "foo@FOOREALM";
+    String hostname = SecurityUtil.getLocalHostName();
     String shouldReplace = service + SecurityUtil.HOSTNAME_PATTERN + realm;
     String replaced = service + hostname + realm;
     verify(shouldReplace, hostname, replaced);
     String shouldNotReplace = service + SecurityUtil.HOSTNAME_PATTERN + "NAME"
         + realm;
     verify(shouldNotReplace, hostname, shouldNotReplace);
-    verify(userPrincipal, hostname, userPrincipal);
-    // testing reverse DNS lookup doesn't happen
-    InetAddress notUsed = Mockito.mock(InetAddress.class);
-    assertEquals(shouldNotReplace, SecurityUtil.getServerPrincipal(
-        shouldNotReplace, notUsed));
-    Mockito.verify(notUsed, Mockito.never()).getCanonicalHostName();
-  }
-  
-  @Test
-  public void testLocalHostNameForNullOrWild() throws Exception {
-    String local = SecurityUtil.getLocalHostName();
-    assertEquals("hdfs/" + local + "@REALM", SecurityUtil.getServerPrincipal(
-        "hdfs/_HOST@REALM", (String) null));
-    assertEquals("hdfs/" + local + "@REALM", SecurityUtil.getServerPrincipal(
-        "hdfs/_HOST@REALM", "0.0.0.0"));
+    verify(shouldNotReplace, shouldNotReplace, shouldNotReplace);
   }
 }