瀏覽代碼

HDFS-14116. [SBN read] Fix class cast error in NNThroughputBenchmark with ObserverReadProxyProvider. Contributed by Chao Sun.

Chao Sun 6 年之前
父節點
當前提交
c3efc299fe

+ 5 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -71,7 +71,6 @@ import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
@@ -2022,18 +2021,17 @@ public class DFSTestUtil {
    * Get the RefreshUserMappingsProtocol RPC proxy for the NN associated with
    * this DFSClient object
    *
-   * @param nameNodeUri the URI of the NN to get a proxy for.
+   * @param nnAddr the address of the NN to get a proxy for.
    *
    * @return the RefreshUserMappingsProtocol RPC proxy associated with this
    * DFSClient object
    */
   @VisibleForTesting
   public static RefreshUserMappingsProtocol getRefreshUserMappingsProtocolProxy(
-      Configuration conf, URI nameNodeUri) throws IOException {
-    final AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
-    return NameNodeProxies.createProxy(conf,
-        nameNodeUri, RefreshUserMappingsProtocol.class,
-        nnFallbackToSimpleAuth).getProxy();
+      Configuration conf, InetSocketAddress nnAddr) throws IOException {
+    return NameNodeProxies.createNonHAProxy(
+        conf, nnAddr, RefreshUserMappingsProtocol.class,
+        UserGroupInformation.getCurrentUser(), false).getProxy();
   }
 
   /**

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -1519,10 +1520,11 @@ public class NNThroughputBenchmark implements Tool {
         nameNodeProto = DFSTestUtil.getNamenodeProtocolProxy(config, nnUri,
             UserGroupInformation.getCurrentUser());
         clientProto = dfs.getClient().getNamenode();
+        InetSocketAddress nnAddr = DFSUtilClient.getNNAddress(nnUri);
         dataNodeProto = new DatanodeProtocolClientSideTranslatorPB(
-            DFSUtilClient.getNNAddress(nnUri), config);
+            nnAddr, config);
         refreshUserMappingsProto =
-            DFSTestUtil.getRefreshUserMappingsProtocolProxy(config, nnUri);
+            DFSTestUtil.getRefreshUserMappingsProtocolProxy(config, nnAddr);
         getBlockPoolId(dfs);
       }
       // run each benchmark