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 bo...@apache.org on 2012/11/06 16:41:12 UTC

svn commit: r1406189 - in /hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common: CHANGES.txt src/main/java/org/apache/hadoop/ipc/Client.java src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java

Author: bobby
Date: Tue Nov  6 15:41:11 2012
New Revision: 1406189

URL: http://svn.apache.org/viewvc?rev=1406189&view=rev
Log:
svn merge -c 1406184 FIXES: HADOOP-9012. IPC Client sends wrong connection context (daryn via bobby)

Modified:
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1406189&r1=1406188&r2=1406189&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt Tue Nov  6 15:41:11 2012
@@ -125,6 +125,8 @@ Release 2.0.3-alpha - Unreleased 
     HADOOP-8713. TestRPCCompatibility fails intermittently with JDK7
     (Trevor Robinson via tgraves)
 
+    HADOOP-9012. IPC Client sends wrong connection context (daryn via bobby)
+
 Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java?rev=1406189&r1=1406188&r2=1406189&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java Tue Nov  6 15:41:11 2012
@@ -221,7 +221,6 @@ public class Client {
   private class Connection extends Thread {
     private InetSocketAddress server;             // server ip:port
     private String serverPrincipal;  // server's krb5 principal name
-    private IpcConnectionContextProto connectionContext;   // connection context
     private final ConnectionId remoteId;                // connection id
     private AuthMethod authMethod; // authentication method
     private Token<? extends TokenIdentifier> token;
@@ -302,9 +301,6 @@ public class Client {
         authMethod = AuthMethod.SIMPLE;
       }
       
-      connectionContext = ProtoUtil.makeIpcConnectionContext(
-          RPC.getProtocolName(protocol), ticket, authMethod);
-      
       if (LOG.isDebugEnabled())
         LOG.debug("Use " + authMethod + " authentication for protocol "
             + protocol.getSimpleName());
@@ -605,11 +601,6 @@ public class Client {
             } else {
               // fall back to simple auth because server told us so.
               authMethod = AuthMethod.SIMPLE;
-              // remake the connectionContext             
-              connectionContext = ProtoUtil.makeIpcConnectionContext(
-                  connectionContext.getProtocol(), 
-                  ProtoUtil.getUgi(connectionContext.getUserInfo()),
-                  authMethod);
             }
           }
         
@@ -620,7 +611,7 @@ public class Client {
             this.in = new DataInputStream(new BufferedInputStream(inStream));
           }
           this.out = new DataOutputStream(new BufferedOutputStream(outStream));
-          writeConnectionContext();
+          writeConnectionContext(remoteId, authMethod);
 
           // update last activity time
           touch();
@@ -742,10 +733,15 @@ public class Client {
     /* Write the connection context header for each connection
      * Out is not synchronized because only the first thread does this.
      */
-    private void writeConnectionContext() throws IOException {
+    private void writeConnectionContext(ConnectionId remoteId,
+                                        AuthMethod authMethod)
+                                            throws IOException {
       // Write out the ConnectionHeader
       DataOutputBuffer buf = new DataOutputBuffer();
-      connectionContext.writeTo(buf);
+      ProtoUtil.makeIpcConnectionContext(
+          RPC.getProtocolName(remoteId.getProtocol()),
+          remoteId.getTicket(),
+          authMethod).writeTo(buf);
       
       // Write out the payload length
       int bufLen = buf.getLength();

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java?rev=1406189&r1=1406188&r2=1406189&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java Tue Nov  6 15:41:11 2012
@@ -29,6 +29,7 @@ import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.Set;
+import java.util.regex.Pattern;
 
 import javax.security.sasl.Sasl;
 
@@ -42,7 +43,6 @@ import org.apache.hadoop.fs.CommonConfig
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.SaslInputStream;
 import org.apache.hadoop.security.SaslRpcClient;
@@ -59,7 +59,7 @@ import org.apache.hadoop.security.token.
 import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.log4j.Level;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 
 /** Unit tests for using Sasl over RPC. */
@@ -76,8 +76,9 @@ public class TestSaslRPC {
   static final String SERVER_PRINCIPAL_2 = "p2/foo@BAR";
   
   private static Configuration conf;
-  @BeforeClass
-  public static void setup() {
+
+  @Before
+  public void setup() {
     conf = new Configuration();
     SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
@@ -187,6 +188,7 @@ public class TestSaslRPC {
   @TokenInfo(TestTokenSelector.class)
   public interface TestSaslProtocol extends TestRPC.TestProtocol {
     public AuthenticationMethod getAuthMethod() throws IOException;
+    public String getAuthUser() throws IOException;
   }
   
   public static class TestSaslImpl extends TestRPC.TestImpl implements
@@ -195,6 +197,10 @@ public class TestSaslRPC {
     public AuthenticationMethod getAuthMethod() throws IOException {
       return UserGroupInformation.getCurrentUser().getAuthenticationMethod();
     }
+    @Override
+    public String getAuthUser() throws IOException {
+      return UserGroupInformation.getCurrentUser().getUserName();
+    }
   }
 
   public static class CustomSecurityInfo extends SecurityInfo {
@@ -261,6 +267,7 @@ public class TestSaslRPC {
 
   @Test
   public void testSecureToInsecureRpc() throws Exception {
+    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.SIMPLE, conf);
     Server server = new RPC.Builder(conf).setProtocol(TestSaslProtocol.class)
         .setInstance(new TestSaslImpl()).setBindAddress(ADDRESS).setPort(0)
         .setNumHandlers(5).setVerbose(true).build();
@@ -448,129 +455,135 @@ public class TestSaslRPC {
     System.out.println("Test is successful.");
   }
 
-  // insecure -> insecure
-  @Test
-  public void testInsecureClientInsecureServer() throws Exception {
-    assertEquals(AuthenticationMethod.SIMPLE,
-                 getAuthMethod(false, false, false));
+  private static Pattern BadToken =
+      Pattern.compile(".*DIGEST-MD5: digest response format violation.*");
+  private static Pattern KrbFailed =
+      Pattern.compile(".*Failed on local exception:.* " +
+                      "Failed to specify server's Kerberos principal name.*");
+  private static Pattern Denied = 
+      Pattern.compile(".*Authorization .* is enabled .*");
+  
+  /*
+   *  simple server
+   */
+  @Test
+  public void testSimpleServer() throws Exception {
+    assertAuthEquals(SIMPLE,    getAuthMethod(SIMPLE,   SIMPLE));
+    // SASL methods are reverted to SIMPLE, but test setup fails
+    assertAuthEquals(KrbFailed, getAuthMethod(KERBEROS, SIMPLE));
   }
 
   @Test
-  public void testInsecureClientInsecureServerWithToken() throws Exception {
-    assertEquals(AuthenticationMethod.TOKEN,
-                 getAuthMethod(false, false, true));
+  public void testSimpleServerWithTokens() throws Exception {
+    // Tokens are ignored because client is reverted to simple
+    assertAuthEquals(SIMPLE, getAuthMethod(SIMPLE,   SIMPLE, true));
+    assertAuthEquals(SIMPLE, getAuthMethod(KERBEROS, SIMPLE, true));
   }
-
-  // insecure -> secure
+    
   @Test
-  public void testInsecureClientSecureServer() throws Exception {
-    RemoteException e = null;
-    try {
-      getAuthMethod(false, true, false);
-    } catch (RemoteException re) {
-      e = re;
-    }
-    assertNotNull(e);
-    assertEquals(AccessControlException.class.getName(), e.getClassName());
+  public void testSimpleServerWithInvalidTokens() throws Exception {
+    // Tokens are ignored because client is reverted to simple
+    assertAuthEquals(SIMPLE, getAuthMethod(SIMPLE,   SIMPLE, false));
+    assertAuthEquals(SIMPLE, getAuthMethod(KERBEROS, SIMPLE, false));
   }
-
+  
+  /*
+   * kerberos server
+   */
   @Test
-  public void testInsecureClientSecureServerWithToken() throws Exception {
-    assertEquals(AuthenticationMethod.TOKEN,
-                 getAuthMethod(false, true, true));
+  public void testKerberosServer() throws Exception {
+    assertAuthEquals(Denied,    getAuthMethod(SIMPLE,   KERBEROS));
+    assertAuthEquals(KrbFailed, getAuthMethod(KERBEROS, KERBEROS));    
   }
 
-  // secure -> secure
   @Test
-  public void testSecureClientSecureServer() throws Exception {
-    /* Should be this when multiple secure auths are supported and we can
-     * dummy one out:
-     *     assertEquals(AuthenticationMethod.SECURE_AUTH_METHOD,
-     *                  getAuthMethod(true, true, false));
-     */
-    try {
-      getAuthMethod(true, true, false);
-    } catch (IOException ioe) {
-      // can't actually test kerberos w/o kerberos...
-      String expectedError = "Failed to specify server's Kerberos principal";
-      String actualError = ioe.getMessage();
-      assertTrue("["+actualError+"] doesn't start with ["+expectedError+"]",
-          actualError.contains(expectedError));
-    }
+  public void testKerberosServerWithTokens() throws Exception {
+    // can use tokens regardless of auth
+    assertAuthEquals(TOKEN, getAuthMethod(SIMPLE,   KERBEROS, true));
+    assertAuthEquals(TOKEN, getAuthMethod(KERBEROS, KERBEROS, true));
   }
 
   @Test
-  public void testSecureClientSecureServerWithToken() throws Exception {
-    assertEquals(AuthenticationMethod.TOKEN,
-                 getAuthMethod(true, true, true));
+  public void testKerberosServerWithInvalidTokens() throws Exception {
+    assertAuthEquals(BadToken, getAuthMethod(SIMPLE,   KERBEROS, false));
+    assertAuthEquals(BadToken, getAuthMethod(KERBEROS, KERBEROS, false));
   }
 
-  // secure -> insecure
-  @Test
-  public void testSecureClientInsecureServerWithToken() throws Exception {
-    assertEquals(AuthenticationMethod.TOKEN,
-                 getAuthMethod(true, false, true));
-  }
 
-  @Test
-  public void testSecureClientInsecureServer() throws Exception {
-    /* Should be this when multiple secure auths are supported and we can
-     * dummy one out:
-     *     assertEquals(AuthenticationMethod.SIMPLE
-     *                  getAuthMethod(true, false, false));
-     */
+  // test helpers
+
+  private String getAuthMethod(
+      final AuthenticationMethod clientAuth,
+      final AuthenticationMethod serverAuth) throws Exception {
     try {
-      getAuthMethod(true, false, false);
-    } catch (IOException ioe) {
-      // can't actually test kerberos w/o kerberos...
-      String expectedError = "Failed to specify server's Kerberos principal";
-      String actualError = ioe.getMessage();
-      assertTrue("["+actualError+"] doesn't start with ["+expectedError+"]",
-          actualError.contains(expectedError));
+      return internalGetAuthMethod(clientAuth, serverAuth, false, false);
+    } catch (Exception e) {
+      return e.toString();
     }
   }
 
-
-  private AuthenticationMethod getAuthMethod(final boolean isSecureClient,
-                                             final boolean isSecureServer,
-                                             final boolean useToken
-                                             
-      ) throws Exception {
+  private String getAuthMethod(
+      final AuthenticationMethod clientAuth,
+      final AuthenticationMethod serverAuth,
+      final boolean useValidToken) throws Exception {
+    try {
+      return internalGetAuthMethod(clientAuth, serverAuth, true, useValidToken);
+    } catch (Exception e) {
+      return e.toString();
+    }
+  }
+  
+  private String internalGetAuthMethod(
+      final AuthenticationMethod clientAuth,
+      final AuthenticationMethod serverAuth,
+      final boolean useToken,
+      final boolean useValidToken) throws Exception {
+    
     Configuration serverConf = new Configuration(conf);
-    SecurityUtil.setAuthenticationMethod(
-        isSecureServer ? KERBEROS : SIMPLE, serverConf);
+    SecurityUtil.setAuthenticationMethod(serverAuth, serverConf);
     UserGroupInformation.setConfiguration(serverConf);
     
     TestTokenSecretManager sm = new TestTokenSecretManager();
     Server server = new RPC.Builder(serverConf).setProtocol(TestSaslProtocol.class)
         .setInstance(new TestSaslImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(5).setVerbose(true).setSecretManager(sm).build();      
+        .setNumHandlers(5).setVerbose(true)
+        .setSecretManager((serverAuth != SIMPLE) ? sm : null)
+        .build();      
     server.start();
 
-    final UserGroupInformation current = UserGroupInformation.getCurrentUser();
+    final UserGroupInformation clientUgi =
+        UserGroupInformation.createRemoteUser(
+            UserGroupInformation.getCurrentUser().getUserName()+"-CLIENT");
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
     if (useToken) {
       TestTokenIdentifier tokenId = new TestTokenIdentifier(
-          new Text(current.getUserName()));
-      Token<TestTokenIdentifier> token =
-          new Token<TestTokenIdentifier>(tokenId, sm);
+          new Text(clientUgi.getUserName()));
+      Token<TestTokenIdentifier> token = useValidToken
+          ? new Token<TestTokenIdentifier>(tokenId, sm)
+          : new Token<TestTokenIdentifier>(
+              tokenId.getBytes(), "bad-password!".getBytes(),
+              tokenId.getKind(), null);
+      
       SecurityUtil.setTokenService(token, addr);
-      current.addToken(token);
+      clientUgi.addToken(token);
     }
 
     final Configuration clientConf = new Configuration(conf);
-    SecurityUtil.setAuthenticationMethod(
-        isSecureClient ? KERBEROS : SIMPLE, clientConf);
+    SecurityUtil.setAuthenticationMethod(clientAuth, clientConf);
     UserGroupInformation.setConfiguration(clientConf);
+    
     try {
-      return current.doAs(new PrivilegedExceptionAction<AuthenticationMethod>() {
+      return clientUgi.doAs(new PrivilegedExceptionAction<String>() {
         @Override
-        public AuthenticationMethod run() throws IOException {
+        public String run() throws IOException {
           TestSaslProtocol proxy = null;
           try {
             proxy = (TestSaslProtocol) RPC.getProxy(TestSaslProtocol.class,
                 TestSaslProtocol.versionID, addr, clientConf);
-            return proxy.getAuthMethod();
+            
+            // make sure the other side thinks we are who we said we are!!!
+            assertEquals(clientUgi.getUserName(), proxy.getAuthUser());
+            return proxy.getAuthMethod().toString();
           } finally {
             if (proxy != null) {
               RPC.stopProxy(proxy);
@@ -582,7 +595,22 @@ public class TestSaslRPC {
       server.stop();
     }
   }
+
+  private static void assertAuthEquals(AuthenticationMethod expect,
+      String actual) {
+    assertEquals(expect.toString(), actual);
+  }
   
+  private static void assertAuthEquals(Pattern expect,
+      String actual) {
+    // this allows us to see the regexp and the value it didn't match
+    if (!expect.matcher(actual).matches()) {
+      assertEquals(expect, actual); // it failed
+    } else {
+      assertTrue(true); // it matched
+    }
+  }
+
   public static void main(String[] args) throws Exception {
     System.out.println("Testing Kerberos authentication over RPC");
     if (args.length != 2) {
@@ -595,5 +623,4 @@ public class TestSaslRPC {
     String keytab = args[1];
     testKerberosRpc(principal, keytab);
   }
-
 }