Forráskód Böngészése

HDFS-17439. Support -nonSuperUser for NNThroughputBenchmark: useful for testing auth frameworks such as Ranger (#6677)

Fateh Singh 10 hónapja
szülő
commit
90024d8cb1

+ 1 - 0
hadoop-common-project/hadoop-common/src/site/markdown/Benchmarking.md

@@ -46,6 +46,7 @@ The following are all supported command options:
 |`-logLevel` | Specify the logging level when the benchmark runs. The default logging level is ERROR. |
 |`-UGCacheRefreshCount` | After every specified number of operations, the benchmark purges the name-node's user group cache. By default the refresh is never called. |
 |`-keepResults` | If specified, do not clean up the name-space after execution. By default the name-space will be removed after test. |
+|`-nonSuperUser` | If specified, non super user can use the tool and can be helpful for bringing authorization time into benchmarking calculations. |
 
 ##### Operations Supported
 

+ 109 - 51
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

@@ -89,10 +89,10 @@ import org.slf4j.event.Level;
 
 /**
  * Main class for a series of name-node benchmarks.
- * 
+ *
  * Each benchmark measures throughput and average execution time 
  * of a specific name-node operation, e.g. file creation or block reports.
- * 
+ *
  * The benchmark does not involve any other hadoop components
  * except for the name-node. Each operation is executed
  * by calling directly the respective name-node method.
@@ -107,7 +107,7 @@ public class NNThroughputBenchmark implements Tool {
       LoggerFactory.getLogger(NNThroughputBenchmark.class);
   private static final int BLOCK_SIZE = 16;
   private static final String GENERAL_OPTIONS_USAGE =
-      "[-keepResults] | [-logLevel L] | [-UGCacheRefreshCount G]";
+      "[-keepResults] | [-logLevel L] | [-UGCacheRefreshCount G] [-nonSuperUser]";
 
   static Configuration config;
   static NameNode nameNode;
@@ -157,7 +157,7 @@ public class NNThroughputBenchmark implements Tool {
 
   /**
    * Base class for collecting operation statistics.
-   * 
+   *
    * Overload this class in order to run statistics for a 
    * specific name-node operation.
    */
@@ -175,6 +175,7 @@ public class NNThroughputBenchmark implements Tool {
     protected long cumulativeTime = 0;    // sum of times for each op
     protected long elapsedTime = 0;       // time from start to finish
     protected boolean keepResults = false;// don't clean base directory on exit
+    protected boolean nonSuperUser = false; // enter/exit safe mode
     protected Level logLevel;             // logging level, ERROR by default
     protected int ugcRefreshCount = 0;    // user group cache refresh count
 
@@ -187,7 +188,7 @@ public class NNThroughputBenchmark implements Tool {
 
     /**
      * Parse command line arguments.
-     * 
+     *
      * @param args arguments
      * @throws IOException
      */
@@ -195,7 +196,7 @@ public class NNThroughputBenchmark implements Tool {
 
     /**
      * Generate inputs for each daemon thread.
-     * 
+     *
      * @param opsPerThread number of inputs for each thread.
      * @throws IOException
      */
@@ -205,7 +206,7 @@ public class NNThroughputBenchmark implements Tool {
      * This corresponds to the arg1 argument of 
      * {@link #executeOp(int, int, String)}, which can have different meanings
      * depending on the operation performed.
-     * 
+     *
      * @param daemonId id of the daemon calling this method
      * @return the argument
      */
@@ -213,7 +214,7 @@ public class NNThroughputBenchmark implements Tool {
 
     /**
      * Execute name-node operation.
-     * 
+     *
      * @param daemonId id of the daemon calling this method.
      * @param inputIdx serial index of the operation called by the deamon.
      * @param arg1 operation specific argument.
@@ -247,7 +248,7 @@ public class NNThroughputBenchmark implements Tool {
           return;
         int tIdx = 0; // thread index < nrThreads
         int opsPerThread[] = new int[numThreads];
-        for(int opsScheduled = 0; opsScheduled < numOpsRequired; 
+        for(int opsScheduled = 0; opsScheduled < numOpsRequired;
                                   opsScheduled += opsPerThread[tIdx++]) {
           // execute  in a separate thread
           opsPerThread[tIdx] = (numOpsRequired-opsScheduled)/(numThreads-tIdx);
@@ -285,14 +286,30 @@ public class NNThroughputBenchmark implements Tool {
     }
 
     void cleanUp() throws IOException {
-      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
-          false);
+      if (!nonSuperUser) {
+        try {
+          clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+                  false);
+        } catch (Exception e){
+          LOG.error("Potentially insufficient permission: try running the tool" +
+                    "with -nonSuperUser argument or login as super user");
+          throw e;
+        }
+      }
       if(!keepResults)
         clientProto.delete(getBaseDir(), true);
       else {
-        clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER,
-            true);
-        clientProto.saveNamespace(0, 0);
+        if (!nonSuperUser) {
+          try {
+            clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER,
+                    true);
+            clientProto.saveNamespace(0, 0);
+          } catch (Exception e){
+            LOG.error("Potentially insufficient permission: try running the tool" +
+                      " with -nonSuperUser argument or login as super user");
+            throw e;
+          }
+        }
       }
     }
     public String getBaseDirName() {
@@ -341,7 +358,7 @@ public class NNThroughputBenchmark implements Tool {
 
     /**
      * Parse first 2 arguments, corresponding to the "-op" option.
-     * 
+     *
      * @param args argument list
      * @return true if operation is all, which means that options not related
      * to this operation should be ignored, or false otherwise, meaning
@@ -358,6 +375,12 @@ public class NNThroughputBenchmark implements Tool {
         args.remove(krIndex);
       }
 
+      int nonSuperUserIndex = args.indexOf("-nonSuperUser");
+      nonSuperUser = (nonSuperUserIndex >= 0);
+      if(nonSuperUser) {
+        args.remove(nonSuperUserIndex);
+      }
+
       int llIndex = args.indexOf("-logLevel");
       if(llIndex >= 0) {
         if(args.size() <= llIndex + 1)
@@ -422,7 +445,7 @@ public class NNThroughputBenchmark implements Tool {
       try {
         benchmarkOne();
       } catch(IOException ex) {
-        LOG.error("StatsDaemon " + daemonId + " failed: \n" 
+        LOG.error("StatsDaemon " + daemonId + " failed: \n"
             + StringUtils.stringifyException(ex));
       }
     }
@@ -499,10 +522,18 @@ public class NNThroughputBenchmark implements Tool {
      * Remove entire benchmark directory.
      */
     @Override
-    long executeOp(int daemonId, int inputIdx, String ignore) 
+    long executeOp(int daemonId, int inputIdx, String ignore)
     throws IOException {
-      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
-          false);
+      if (!nonSuperUser) {
+        try{
+          clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+                  false);
+        } catch (Exception e){
+          LOG.error("Potentially insufficient permission: try running the tool" +
+                    " with -nonSuperUser argument or login as super user");
+          throw e;
+        }
+      }
       long start = Time.now();
       clientProto.delete(getBaseDirName(), true);
       long end = Time.now();
@@ -519,7 +550,7 @@ public class NNThroughputBenchmark implements Tool {
 
   /**
    * File creation statistics.
-   * 
+   *
    * Each thread creates the same (+ or -1) number of files.
    * File names are pre-generated during initialization.
    * The created files do not have blocks.
@@ -578,9 +609,17 @@ public class NNThroughputBenchmark implements Tool {
 
     @Override
     void generateInputs(int[] opsPerThread) throws IOException {
-      assert opsPerThread.length == numThreads : "Error opsPerThread.length"; 
-      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
-          false);
+      assert opsPerThread.length == numThreads : "Error opsPerThread.length";
+      if (!nonSuperUser) {
+        try{
+          clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+              false);
+        } catch (Exception e){
+          LOG.error("Potentially insufficient permission: try running the tool" +
+                    "with -nonSuperUser argument or login as super user");
+          throw e;
+        }
+      }
       // int generatedFileIdx = 0;
       LOG.info("Generate " + numOpsRequired + " intputs for " + getOpName());
       LOG.info("basedir: " + getBaseDir());
@@ -614,7 +653,7 @@ public class NNThroughputBenchmark implements Tool {
      * Do file create.
      */
     @Override
-    long executeOp(int daemonId, int inputIdx, String clientName) 
+    long executeOp(int daemonId, int inputIdx, String clientName)
     throws IOException {
       long start = Time.now();
       clientProto.create(fileNames[daemonId][inputIdx],
@@ -695,8 +734,16 @@ public class NNThroughputBenchmark implements Tool {
     @Override
     void generateInputs(int[] opsPerThread) throws IOException {
       assert opsPerThread.length == numThreads : "Error opsPerThread.length";
-      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
-          false);
+      if (!nonSuperUser) {
+        try {
+          clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+                  false);
+        } catch (Exception e){
+          LOG.error("Potentially insufficient permission: try running the tool" +
+                    " with -nonSuperUser argument or login as super user");
+          throw e;
+        }
+      }
       LOG.info("Generate " + numOpsRequired + " inputs for " + getOpName());
       dirPaths = new String[numThreads][];
       try {
@@ -750,14 +797,14 @@ public class NNThroughputBenchmark implements Tool {
 
   /**
    * Open file statistics.
-   * 
-   * Measure how many open calls (getBlockLocations()) 
+   *
+   * Measure how many open calls (getBlockLocations())
    * the name-node can handle per second.
    */
   class OpenFileStats extends CreateFileStats {
     // Operation types
     static final String OP_OPEN_NAME = "open";
-    static final String OP_USAGE_ARGS = 
+    static final String OP_USAGE_ARGS =
         " [-threads T] [-files N] [-blockSize S] [-filesPerDir P]"
         + " [-useExisting] [-baseDirName D]";
     static final String OP_OPEN_USAGE = 
@@ -796,13 +843,16 @@ public class NNThroughputBenchmark implements Tool {
           String.valueOf(nameGenerator.getFilesPerDirectory()),
           "-baseDirName", getBaseDirName(),
           "-close"};
-      CreateFileStats opCreate =  new CreateFileStats(Arrays.asList(createArgs));
-
+      List<String> createArgsList = new ArrayList<String>(Arrays.asList(createArgs));
+      if (this.nonSuperUser){
+        createArgsList.add("-nonSuperUser");
+      }
+      CreateFileStats opCreate =  new CreateFileStats(createArgsList);
       if(!useExisting) {  // create files if they were not created before
         opCreate.benchmark();
         LOG.info("Created " + numOpsRequired + " files.");
       } else {
-        LOG.info("useExisting = true. Assuming " 
+        LOG.info("useExisting = true. Assuming "
             + numOpsRequired + " files have been created before.");
       }
       // use the same files for open
@@ -820,7 +870,7 @@ public class NNThroughputBenchmark implements Tool {
      * Do file open.
      */
     @Override
-    long executeOp(int daemonId, int inputIdx, String ignore) 
+    long executeOp(int daemonId, int inputIdx, String ignore)
     throws IOException {
       long start = Time.now();
       clientProto.getBlockLocations(fileNames[daemonId][inputIdx], 0L,
@@ -832,13 +882,13 @@ public class NNThroughputBenchmark implements Tool {
 
   /**
    * Delete file statistics.
-   * 
+   *
    * Measure how many delete calls the name-node can handle per second.
    */
   class DeleteFileStats extends OpenFileStats {
     // Operation types
     static final String OP_DELETE_NAME = "delete";
-    static final String OP_DELETE_USAGE = 
+    static final String OP_DELETE_USAGE =
       "-op " + OP_DELETE_NAME + OP_USAGE_ARGS;
 
     DeleteFileStats(List<String> args) {
@@ -851,7 +901,7 @@ public class NNThroughputBenchmark implements Tool {
     }
 
     @Override
-    long executeOp(int daemonId, int inputIdx, String ignore) 
+    long executeOp(int daemonId, int inputIdx, String ignore)
     throws IOException {
       long start = Time.now();
       clientProto.delete(fileNames[daemonId][inputIdx], false);
@@ -909,13 +959,13 @@ public class NNThroughputBenchmark implements Tool {
 
   /**
    * List file status statistics.
-   * 
+   *
    * Measure how many get-file-status calls the name-node can handle per second.
    */
   class FileStatusStats extends OpenFileStats {
     // Operation types
     static final String OP_FILE_STATUS_NAME = "fileStatus";
-    static final String OP_FILE_STATUS_USAGE = 
+    static final String OP_FILE_STATUS_USAGE =
       "-op " + OP_FILE_STATUS_NAME + OP_USAGE_ARGS;
 
     FileStatusStats(List<String> args) {
@@ -928,7 +978,7 @@ public class NNThroughputBenchmark implements Tool {
     }
 
     @Override
-    long executeOp(int daemonId, int inputIdx, String ignore) 
+    long executeOp(int daemonId, int inputIdx, String ignore)
     throws IOException {
       long start = Time.now();
       clientProto.getFileInfo(fileNames[daemonId][inputIdx]);
@@ -939,13 +989,13 @@ public class NNThroughputBenchmark implements Tool {
 
   /**
    * Rename file statistics.
-   * 
+   *
    * Measure how many rename calls the name-node can handle per second.
    */
   class RenameFileStats extends OpenFileStats {
     // Operation types
     static final String OP_RENAME_NAME = "rename";
-    static final String OP_RENAME_USAGE = 
+    static final String OP_RENAME_USAGE =
       "-op " + OP_RENAME_NAME + OP_USAGE_ARGS;
 
     protected String[][] destNames;
@@ -972,7 +1022,7 @@ public class NNThroughputBenchmark implements Tool {
     }
 
     @Override
-    long executeOp(int daemonId, int inputIdx, String ignore) 
+    long executeOp(int daemonId, int inputIdx, String ignore)
     throws IOException {
       long start = Time.now();
       clientProto.rename(fileNames[daemonId][inputIdx],
@@ -988,10 +1038,10 @@ public class NNThroughputBenchmark implements Tool {
   private static class TinyDatanode implements Comparable<String> {
     private static final long DF_CAPACITY = 100*1024*1024;
     private static final long DF_USED = 0;
-    
+
     NamespaceInfo nsInfo;
     DatanodeRegistration dnRegistration;
-    DatanodeStorage storage; //only one storage 
+    DatanodeStorage storage; //only one storage
     final List<BlockReportReplica> blocks;
     int nrBlocks; // actual number of blocks
     BlockListAsLongs blockReportList;
@@ -1124,7 +1174,7 @@ public class NNThroughputBenchmark implements Tool {
      * Just report on behalf of the other data-node
      * that the blocks have been received.
      */
-    private int transferBlocks( Block blocks[], 
+    private int transferBlocks( Block blocks[],
                                 DatanodeInfo xferTargets[][],
                                 String targetStorageIDs[][]
                               ) throws IOException {
@@ -1152,7 +1202,7 @@ public class NNThroughputBenchmark implements Tool {
 
   /**
    * Block report statistics.
-   * 
+   *
    * Each thread here represents its own data-node.
    * Data-nodes send the same block report each time.
    * The block report may contain missing or non-existing blocks.
@@ -1224,7 +1274,7 @@ public class NNThroughputBenchmark implements Tool {
     @Override
     void generateInputs(int[] ignore) throws IOException {
       int nrDatanodes = getNumDatanodes();
-      int nrBlocks = (int)Math.ceil((double)blocksPerReport * nrDatanodes 
+      int nrBlocks = (int)Math.ceil((double)blocksPerReport * nrDatanodes
                                     / replication);
       int nrFiles = (int)Math.ceil((double)nrBlocks / blocksPerFile);
       datanodes = new TinyDatanode[nrDatanodes];
@@ -1235,13 +1285,21 @@ public class NNThroughputBenchmark implements Tool {
         datanodes[idx].sendHeartbeat();
       }
 
-      // create files 
+      // create files
       LOG.info("Creating " + nrFiles + " files with " + blocksPerFile + " blocks each.");
       FileNameGenerator nameGenerator;
       nameGenerator = new FileNameGenerator(getBaseDir(), 100);
       String clientName = getClientName(007);
-      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
-          false);
+      if (!nonSuperUser) {
+        try {
+          clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+                  false);
+        } catch (Exception e){
+          LOG.error("Potentially insufficient permission: try running the tool" +
+                    " with -nonSuperUser argument or login as super user");
+          throw e;
+        }
+      }
       for(int idx=0; idx < nrFiles; idx++) {
         String fileName = nameGenerator.getNextFileName("ThroughputBench");
         clientProto.create(fileName, FsPermission.getDefault(), clientName,
@@ -1375,7 +1433,7 @@ public class NNThroughputBenchmark implements Tool {
    */
   class ReplicationStats extends OperationStatsBase {
     static final String OP_REPLICATION_NAME = "replication";
-    static final String OP_REPLICATION_USAGE = 
+    static final String OP_REPLICATION_USAGE =
         "-op replication [-datanodes T] [-nodesToDecommission D] " +
         "[-nodeReplicationLimit C] [-totalBlocks B] [-blockSize S] "
         + "[-replication R] [-baseDirName D]";

+ 24 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java

@@ -100,7 +100,30 @@ public class TestNNThroughputBenchmark {
       }
     }
   }
-
+  /**
+   * This test runs {@link NNThroughputBenchmark} against a mini DFS cluster with
+   * nonSuperUser option (useful when testing any authorization framework e.g.
+   * Ranger since only super user e.g. hdfs can enter/exit safemode
+   * but any request from super user is not sent for authorization).
+   */
+  @Test(timeout = 120000)
+  public void testNNThroughputAgainstRemoteNNNonSuperUser() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 16);
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      cluster.waitActive();
+      final Configuration benchConf = new HdfsConfiguration();
+      benchConf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 16);
+      FileSystem.setDefaultUri(benchConf, cluster.getURI());
+      NNThroughputBenchmark.runBenchmark(benchConf, new String[]{"-op", "all", "-nonSuperUser"});
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
   /**
    * This test runs {@link NNThroughputBenchmark} against a mini DFS cluster
    * with explicit -fs option.