瀏覽代碼

HADOOP-12670. Fix TestNetUtils and TestSecurityUtil when localhost is ipv6 only. Contributed by Elliott Neil Clark And Hemanth Boyina.

Brahma Reddy Battula 3 年之前
父節點
當前提交
3133386ac4

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java

@@ -768,7 +768,7 @@ public class NetUtils {
     if (InetAddressUtils.isIPv6Address(hostName)) {
       return "[" + hostName + "]:" + addr.getPort();
     }
-    return hostName + ":" + addr.getPort();
+    return hostName.toLowerCase() + ":" + addr.getPort();
   }
   /**
    * Compose a "ip:port" string from the InetSocketAddress.

+ 3 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java

@@ -445,7 +445,7 @@ public final class SecurityUtil {
     if (token != null) {
       token.setService(service);
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Acquired token "+token);  // Token#toString() prints service
+        LOG.debug("Acquired token " + token);  // Token#toString() prints service
       }
     } else {
       LOG.warn("Failed to get token for service "+service);
@@ -459,18 +459,15 @@ public final class SecurityUtil {
    *          hadoop.security.token.service.use_ip
    */
   public static Text buildTokenService(InetSocketAddress addr) {
-    String host = null;
     if (useIpForTokenService) {
       if (addr.isUnresolved()) { // host has no ip address
         throw new IllegalArgumentException(
             new UnknownHostException(addr.getHostName())
         );
       }
-      host = addr.getAddress().getHostAddress();
-    } else {
-      host = StringUtils.toLowerCase(addr.getHostName());
+      return new Text(NetUtils.getIPPortString(addr));
     }
-    return new Text(host + ":" + addr.getPort());
+    return new Text(NetUtils.getHostPortString(addr));
   }
 
   /**

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java

@@ -125,10 +125,10 @@ public class DefaultImpersonationProvider implements ImpersonationProvider {
           + " is not allowed to impersonate " + user.getUserName());
     }
 
-    MachineList MachineList = proxyHosts.get(
+    MachineList machineList = proxyHosts.get(
         getProxySuperuserIpConfKey(realUser.getShortUserName()));
 
-    if(MachineList == null || !MachineList.includes(remoteAddress)) {
+    if(machineList == null || !machineList.includes(remoteAddress)) {
       throw new AuthorizationException("Unauthorized connection for super-user: "
           + realUser.getUserName() + " from IP " + remoteAddress);
     }

+ 21 - 10
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java

@@ -591,13 +591,19 @@ public class TestNetUtils {
     return addr;
   }
 
-  private void
-  verifyInetAddress(InetAddress addr, String host, String ip) {
+  private void verifyInetAddress(InetAddress addr, String host, String... ips) {
     assertNotNull(addr);
     assertEquals(host, addr.getHostName());
-    assertEquals(ip, addr.getHostAddress());
+
+    boolean found = false;
+    for (String ip : ips) {
+      found |= ip.equals(addr.getHostAddress());
+    }
+    assertTrue("Expected addr.getHostAddress[" + addr.getHostAddress()
+        + "]  to be one of " + StringUtils.join(ips, ","), found);
   }
-  
+
+
   @Test
   public void testResolverUnqualified() {
     String host = "host";
@@ -627,12 +633,13 @@ public class TestNetUtils {
   }
   
   // localhost
-  
+
   @Test
   public void testResolverLoopback() {
     String host = "Localhost";
     InetAddress addr = verifyResolve(host); // no lookup should occur
-    verifyInetAddress(addr, "Localhost", "127.0.0.1");
+    verifyInetAddress(addr, "Localhost", "127.0.0.1", IPV6_LOOPBACK_LONG_STRING,
+        IPV6_LOOPBACK_SHORT_STRING);
   }
 
   @Test
@@ -737,11 +744,14 @@ public class TestNetUtils {
     // when ipaddress is normalized, same address is expected in return
     assertEquals(summary, hosts.get(0), normalizedHosts.get(0));
     // for normalizing a resolvable hostname, resolved ipaddress is expected in return
+
     assertFalse("Element 1 equal "+ summary,
         normalizedHosts.get(1).equals(hosts.get(1)));
-    assertEquals(summary, hosts.get(0), normalizedHosts.get(1));
-    // this address HADOOP-8372: when normalizing a valid resolvable hostname start with numeric, 
-    // its ipaddress is expected to return
+    assertTrue("Should get the localhost address back",
+        normalizedHosts.get(1).equals(hosts.get(0)) || normalizedHosts.get(1)
+            .equals(IPV6_LOOPBACK_LONG_STRING));
+    // this address HADOOP-8372: when normalizing a valid resolvable hostname
+    // start with numeric, its ipaddress is expected to return
     assertFalse("Element 2 equal " + summary,
         normalizedHosts.get(2).equals(hosts.get(2)));
     // return the same hostname after normalizing a irresolvable hostname.
@@ -790,7 +800,8 @@ public class TestNetUtils {
 
     InetSocketAddress addr = NetUtils.createSocketAddr(defaultAddr);
     conf.setSocketAddr("myAddress", addr);
-    assertEquals(defaultAddr.trim(), NetUtils.getHostPortString(addr));
+    assertTrue("Trim should have been called on ipv6 hostname",
+        defaultAddr.trim().equalsIgnoreCase(NetUtils.getHostPortString(addr)));
   }
 
   @Test

+ 0 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java

@@ -25,7 +25,6 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine2;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.TestRpcBase;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
@@ -361,7 +360,6 @@ public class TestDoAsEffectiveUser extends TestRpcBase {
   public void testProxyWithToken() throws Exception {
     final Configuration conf = new Configuration(masterConf);
     TestTokenSecretManager sm = new TestTokenSecretManager();
-    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
     RPC.setProtocolEngine(conf, TestRpcService.class,
         ProtobufRpcEngine2.class);
     UserGroupInformation.setConfiguration(conf);
@@ -408,7 +406,6 @@ public class TestDoAsEffectiveUser extends TestRpcBase {
   public void testTokenBySuperUser() throws Exception {
     TestTokenSecretManager sm = new TestTokenSecretManager();
     final Configuration newConf = new Configuration(masterConf);
-    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, newConf);
     // Set RPC engine to protobuf RPC engine
     RPC.setProtocolEngine(newConf, TestRpcService.class,
         ProtobufRpcEngine2.class);

+ 15 - 10
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java

@@ -182,16 +182,12 @@ public class TestSecurityUtil {
     conf.setBoolean(
         CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP, true);
     SecurityUtil.setConfiguration(conf);
-    assertEquals("127.0.0.1:123",
-        SecurityUtil.buildTokenService(new InetSocketAddress("LocalHost", 123)).toString()
-    );
-    assertEquals("127.0.0.1:123",
-        SecurityUtil.buildTokenService(new InetSocketAddress("127.0.0.1", 123)).toString()
-    );
-    // what goes in, comes out
-    assertEquals("127.0.0.1:123",
-        SecurityUtil.buildTokenService(NetUtils.createSocketAddr("127.0.0.1", 123)).toString()
-    );
+    assertOneOf(SecurityUtil
+        .buildTokenService(NetUtils.createSocketAddrForHost("LocalHost", 123))
+        .toString(), "127.0.0.1:123", "[0:0:0:0:0:0:0:1]:123");
+    assertOneOf(SecurityUtil
+        .buildTokenService(NetUtils.createSocketAddrForHost("127.0.0.1", 123))
+        .toString(), "127.0.0.1:123", "[0:0:0:0:0:0:0:1]:123");
   }
 
   @Test
@@ -496,4 +492,13 @@ public class TestSecurityUtil {
         ZK_AUTH_VALUE.toCharArray());
     provider.flush();
   }
+
+  private void assertOneOf(String value, String... expected) {
+    boolean found = false;
+    for (String ip : expected) {
+      found |= ip.equals(value);
+    }
+    assertTrue("Expected value [" + value + "]  to be one of " + StringUtils
+        .join(",", expected), found);
+  }
 }