浏览代码

HDFS-12967. NNBench should support multi-cluster access. Contributed by Chen Zhang.

Signed-off-by: Wei-Chiu Chuang <weichiu@apache.org>
Chen Zhang 5 年之前
父节点
当前提交
8a59cd1b8a

+ 7 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java

@@ -117,7 +117,7 @@ public class NNBench extends Configured implements Tool {
    * @throws IOException on error
    */
   private void cleanupBeforeTestrun() throws IOException {
-    FileSystem tempFS = FileSystem.get(getConf());
+    FileSystem tempFS = FileSystem.get(new Path(baseDir).toUri(), getConf());
     
     // Delete the data directory only if it is the create/write operation
     if (operation.equals(OP_CREATE_WRITE)) {
@@ -193,7 +193,8 @@ public class NNBench extends Configured implements Tool {
       "\t-replicationFactorPerFile <Replication factor for the files." +
         " default is 1. This is not mandatory>\n" +
       "\t-baseDir <base DFS path. default is /benchmarks/NNBench. " +
-      "This is not mandatory>\n" +
+        "Supports cross-cluster access by using full path with schema and " +
+        "cluster. This is not mandatory>\n" +
       "\t-readFileAfterOpen <true or false. if true, it reads the file and " +
       "reports the average time to read. This is valid with the open_read " +
       "operation. default is false. This is not mandatory>\n" +
@@ -305,7 +306,7 @@ public class NNBench extends Configured implements Tool {
    * @throws IOException on error
    */
   private int analyzeResults() throws IOException {
-    final FileSystem fs = FileSystem.get(getConf());
+    final FileSystem fs = FileSystem.get(new Path(baseDir).toUri(), getConf());
     Path reduceDir = new Path(baseDir, OUTPUT_DIR_NAME);
 
     long totalTimeAL1 = 0l;
@@ -642,9 +643,10 @@ public class NNBench extends Configured implements Tool {
      */
     public void configure(JobConf conf) {
       setConf(conf);
-      
+
       try {
-        filesystem = FileSystem.get(conf);
+        String dir = conf.get("test.nnbench.basedir");
+        filesystem = FileSystem.get(new Path(dir).toUri(), conf);
       } catch(Exception e) {
         throw new RuntimeException("Cannot get file system.", e);
       }

+ 24 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/TestNNBench.java

@@ -27,6 +27,7 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
@@ -73,12 +74,32 @@ public class TestNNBench extends HadoopTestCase {
         getFileSystem().exists(renamedPath));
   }
 
-  private void runNNBench(Configuration conf, String operation)
+  @Test(timeout = 30000)
+  public void testNNBenchCrossCluster() throws Exception {
+    MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(new JobConf())
+            .numDataNodes(1).build();
+    dfsCluster.waitClusterUp();
+    String nnAddress = dfsCluster.getNameNode(0).getHostAndPort();
+    String baseDir = "hdfs://" + nnAddress + BASE_DIR;
+    runNNBench(createJobConf(), "create_write", baseDir);
+
+    Path path = new Path(BASE_DIR + "/data/file_0_0");
+    assertTrue("create_write should create the file",
+            dfsCluster.getFileSystem().exists(path));
+    dfsCluster.shutdown();
+  }
+
+  private void runNNBench(Configuration conf, String operation, String baseDir)
       throws Exception {
-    String[] genArgs = { "-operation", operation, "-baseDir", BASE_DIR,
-        "-startTime", "" + (Time.now() / 1000 + 3) };
+    String[] genArgs = {"-operation", operation, "-baseDir", baseDir,
+        "-startTime", "" + (Time.now() / 1000 + 3), "-blockSize", "1024"};
 
     assertEquals(0, ToolRunner.run(conf, new NNBench(), genArgs));
   }
 
+  private void runNNBench(Configuration conf, String operation)
+      throws Exception {
+    runNNBench(conf, operation, BASE_DIR);
+  }
+
 }