瀏覽代碼

HADOOP-14062. ApplicationMasterProtocolPBClientImpl.allocate fails with EOFException when RPC privacy is enabled. Contributed by Steven Rand

Jian He 8 年之前
父節點
當前提交
66e062e537

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -71,7 +71,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
-import static org.apache.hadoop.ipc.RpcConstants.CONNECTION_CONTEXT_CALL_ID;
 import static org.apache.hadoop.ipc.RpcConstants.PING_CALL_ID;
 
 /** A client for an IPC service.  IPC calls take a single {@link Writable} as a
@@ -1767,7 +1766,9 @@ public class Client implements AutoCloseable {
     }
 
     void setSaslClient(SaslRpcClient client) throws IOException {
-      setInputStream(client.getInputStream(in));
+      // Wrap the input stream in a BufferedInputStream to fill the buffer
+      // before reading its length (HADOOP-14062).
+      setInputStream(new BufferedInputStream(client.getInputStream(in)));
       setOutputStream(client.getOutputStream(out));
     }
 

+ 29 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java

@@ -39,7 +39,7 @@ import java.util.Set;
 import java.util.TreeSet;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -133,6 +133,11 @@ public class TestAMRMClient {
     // set the minimum allocation so that resource decrease can go under 1024
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512);
     conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
+    createClientAndCluster(conf);
+  }
+
+  private static void createClientAndCluster(Configuration conf)
+      throws Exception {
     yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
     yarnCluster.init(conf);
     yarnCluster.start();
@@ -656,6 +661,29 @@ public class TestAMRMClient {
 
   @Test (timeout=60000)
   public void testAMRMClient() throws YarnException, IOException {
+    registerAndAllocate();
+  }
+
+  @Test (timeout=60000)
+  public void testAMRMClientWithSaslEncryption() throws Exception {
+    conf.set(CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION, "privacy");
+    // we have to create a new instance of MiniYARNCluster to avoid SASL qop
+    // mismatches between client and server
+    tearDown();
+    createClientAndCluster(conf);
+    startApp();
+    registerAndAllocate();
+
+    // recreate the original MiniYARNCluster and YarnClient for other tests
+    conf.unset(CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION);
+    tearDown();
+    createClientAndCluster(conf);
+    // unless we start an application the cancelApp() method will fail when
+    // it runs after this test
+    startApp();
+  }
+
+  private void registerAndAllocate() throws YarnException, IOException {
     AMRMClient<ContainerRequest> amClient = null;
     try {
       // start am rm client