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 ji...@apache.org on 2015/01/13 02:35:27 UTC

hadoop git commit: HDFS-7600. Refine hdfs admin classes to reuse common code. Contributed by Jing Zhao.

Repository: hadoop
Updated Branches:
  refs/heads/trunk f761bd8fe -> 6f3a63a41


HDFS-7600. Refine hdfs admin classes to reuse common code. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6f3a63a4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6f3a63a4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6f3a63a4

Branch: refs/heads/trunk
Commit: 6f3a63a41b90157c3e46ea20ca6170b854ea902e
Parents: f761bd8
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Jan 12 17:32:33 2015 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Mon Jan 12 17:32:33 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../apache/hadoop/hdfs/tools/AdminHelper.java   | 192 ++++++++++++++
 .../apache/hadoop/hdfs/tools/CacheAdmin.java    | 264 +++++--------------
 .../apache/hadoop/hdfs/tools/CryptoAdmin.java   | 130 +--------
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   | 148 ++---------
 5 files changed, 288 insertions(+), 448 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f3a63a4/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 487b209..cefdc16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -496,6 +496,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-7598. Remove dependency on old version of guava in
     TestDFSClientCache#testEviction. (Sangjin Lee via Colin P. McCabe)
 
+    HDFS-7600. Refine hdfs admin classes to reuse common code. (jing9)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f3a63a4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java
new file mode 100644
index 0000000..153fb36
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.tools.TableListing;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Helper methods for CacheAdmin/CryptoAdmin/StoragePolicyAdmin
+ */
+public class AdminHelper {
+  /**
+   * Maximum length for printed lines
+   */
+  static final int MAX_LINE_WIDTH = 80;
+  static final String HELP_COMMAND_NAME = "-help";
+
+  static DistributedFileSystem getDFS(Configuration conf)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    if (!(fs instanceof DistributedFileSystem)) {
+      throw new IllegalArgumentException("FileSystem " + fs.getUri() +
+          " is not an HDFS file system");
+    }
+    return (DistributedFileSystem)fs;
+  }
+
+  /**
+   * NN exceptions contain the stack trace as part of the exception message.
+   * When it's a known error, pretty-print the error and squish the stack trace.
+   */
+  static String prettifyException(Exception e) {
+    return e.getClass().getSimpleName() + ": "
+        + e.getLocalizedMessage().split("\n")[0];
+  }
+
+  static TableListing getOptionDescriptionListing() {
+    return new TableListing.Builder()
+        .addField("").addField("", true)
+        .wrapWidth(MAX_LINE_WIDTH).hideHeaders().build();
+  }
+
+  /**
+   * Parses a time-to-live value from a string
+   * @return The ttl in milliseconds
+   * @throws IOException if it could not be parsed
+   */
+  static Long parseTtlString(String maxTtlString) throws IOException {
+    Long maxTtl = null;
+    if (maxTtlString != null) {
+      if (maxTtlString.equalsIgnoreCase("never")) {
+        maxTtl = CachePoolInfo.RELATIVE_EXPIRY_NEVER;
+      } else {
+        maxTtl = DFSUtil.parseRelativeTime(maxTtlString);
+      }
+    }
+    return maxTtl;
+  }
+
+  static Long parseLimitString(String limitString) {
+    Long limit = null;
+    if (limitString != null) {
+      if (limitString.equalsIgnoreCase("unlimited")) {
+        limit = CachePoolInfo.LIMIT_UNLIMITED;
+      } else {
+        limit = Long.parseLong(limitString);
+      }
+    }
+    return limit;
+  }
+
+  static Command determineCommand(String commandName, Command[] commands) {
+    Preconditions.checkNotNull(commands);
+    if (HELP_COMMAND_NAME.equals(commandName)) {
+      return new HelpCommand(commands);
+    }
+    for (Command command : commands) {
+      if (command.getName().equals(commandName)) {
+        return command;
+      }
+    }
+    return null;
+  }
+
+  static void printUsage(boolean longUsage, String toolName,
+      Command[] commands) {
+    Preconditions.checkNotNull(commands);
+    System.err.println("Usage: bin/hdfs " + toolName + " [COMMAND]");
+    final HelpCommand helpCommand = new HelpCommand(commands);
+    for (AdminHelper.Command command : commands) {
+      if (longUsage) {
+        System.err.print(command.getLongUsage());
+      } else {
+        System.err.print("          " + command.getShortUsage());
+      }
+    }
+    System.err.print(longUsage ? helpCommand.getLongUsage() :
+        ("          " + helpCommand.getShortUsage()));
+    System.err.println();
+  }
+
+  interface Command {
+    String getName();
+    String getShortUsage();
+    String getLongUsage();
+    int run(Configuration conf, List<String> args) throws IOException;
+  }
+
+  static class HelpCommand implements Command {
+    private final Command[] commands;
+
+    public HelpCommand(Command[] commands) {
+      Preconditions.checkNotNull(commands != null);
+      this.commands = commands;
+    }
+
+    @Override
+    public String getName() {
+      return HELP_COMMAND_NAME;
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[-help <command-name>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      final TableListing listing = AdminHelper.getOptionDescriptionListing();
+      listing.addRow("<command-name>", "The command for which to get " +
+          "detailed help. If no command is specified, print detailed help for " +
+          "all commands");
+      return getShortUsage() + "\n" +
+          "Get detailed help about a command.\n\n" +
+          listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      if (args.size() == 0) {
+        for (AdminHelper.Command command : commands) {
+          System.err.println(command.getLongUsage());
+        }
+        return 0;
+      }
+      if (args.size() != 1) {
+        System.out.println("You must give exactly one argument to -help.");
+        return 0;
+      }
+      final String commandName = args.get(0);
+      // prepend a dash to match against the command names
+      final AdminHelper.Command command = AdminHelper
+          .determineCommand("-" + commandName, commands);
+      if (command == null) {
+        System.err.print("Unknown command '" + commandName + "'.\n");
+        System.err.print("Valid help command names are:\n");
+        String separator = "";
+        for (AdminHelper.Command c : commands) {
+          System.err.print(separator + c.getName().substring(1));
+          separator = ", ";
+        }
+        System.err.print("\n");
+        return 1;
+      }
+      System.err.print(command.getLongUsage());
+      return 0;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f3a63a4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
index dbe2284..6888ea8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -53,11 +52,6 @@ import com.google.common.base.Joiner;
 @InterfaceAudience.Private
 public class CacheAdmin extends Configured implements Tool {
 
-  /**
-   * Maximum length for printed lines
-   */
-  private static final int MAX_LINE_WIDTH = 80;
-
   public CacheAdmin() {
     this(null);
   }
@@ -69,16 +63,17 @@ public class CacheAdmin extends Configured implements Tool {
   @Override
   public int run(String[] args) throws IOException {
     if (args.length == 0) {
-      printUsage(false);
+      AdminHelper.printUsage(false, "cacheadmin", COMMANDS);
       return 1;
     }
-    Command command = determineCommand(args[0]);
+    AdminHelper.Command command = AdminHelper.determineCommand(args[0],
+        COMMANDS);
     if (command == null) {
       System.err.println("Can't understand command '" + args[0] + "'");
       if (!args[0].startsWith("-")) {
         System.err.println("Command names must start with dashes.");
       }
-      printUsage(false);
+      AdminHelper.printUsage(false, "cacheadmin", COMMANDS);
       return 1;
     }
     List<String> argsList = new LinkedList<String>();
@@ -88,7 +83,7 @@ public class CacheAdmin extends Configured implements Tool {
     try {
       return command.run(getConf(), argsList);
     } catch (IllegalArgumentException e) {
-      System.err.println(prettifyException(e));
+      System.err.println(AdminHelper.prettifyException(e));
       return -1;
     }
   }
@@ -98,64 +93,9 @@ public class CacheAdmin extends Configured implements Tool {
     System.exit(cacheAdmin.run(argsArray));
   }
 
-  private static DistributedFileSystem getDFS(Configuration conf)
+  private static CacheDirectiveInfo.Expiration parseExpirationString(String ttlString)
       throws IOException {
-    FileSystem fs = FileSystem.get(conf);
-    if (!(fs instanceof DistributedFileSystem)) {
-      throw new IllegalArgumentException("FileSystem " + fs.getUri() + 
-      " is not an HDFS file system");
-    }
-    return (DistributedFileSystem)fs;
-  }
-
-  /**
-   * NN exceptions contain the stack trace as part of the exception message.
-   * When it's a known error, pretty-print the error and squish the stack trace.
-   */
-  private static String prettifyException(Exception e) {
-    return e.getClass().getSimpleName() + ": "
-        + e.getLocalizedMessage().split("\n")[0];
-  }
-
-  private static TableListing getOptionDescriptionListing() {
-    TableListing listing = new TableListing.Builder()
-    .addField("").addField("", true)
-    .wrapWidth(MAX_LINE_WIDTH).hideHeaders().build();
-    return listing;
-  }
-
-  /**
-   * Parses a time-to-live value from a string
-   * @return The ttl in milliseconds
-   * @throws IOException if it could not be parsed
-   */
-  private static Long parseTtlString(String maxTtlString) throws IOException {
-    Long maxTtl = null;
-    if (maxTtlString != null) {
-      if (maxTtlString.equalsIgnoreCase("never")) {
-        maxTtl = CachePoolInfo.RELATIVE_EXPIRY_NEVER;
-      } else {
-        maxTtl = DFSUtil.parseRelativeTime(maxTtlString);
-      }
-    }
-    return maxTtl;
-  }
-
-  private static Long parseLimitString(String limitString) {
-    Long limit = null;
-    if (limitString != null) {
-      if (limitString.equalsIgnoreCase("unlimited")) {
-        limit = CachePoolInfo.LIMIT_UNLIMITED;
-      } else {
-        limit = Long.parseLong(limitString);
-      }
-    }
-    return limit;
-  }
-
-  private static Expiration parseExpirationString(String ttlString)
-      throws IOException {
-    Expiration ex = null;
+    CacheDirectiveInfo.Expiration ex = null;
     if (ttlString != null) {
       if (ttlString.equalsIgnoreCase("never")) {
         ex = CacheDirectiveInfo.Expiration.NEVER;
@@ -167,14 +107,8 @@ public class CacheAdmin extends Configured implements Tool {
     return ex;
   }
 
-  interface Command {
-    String getName();
-    String getShortUsage();
-    String getLongUsage();
-    int run(Configuration conf, List<String> args) throws IOException;
-  }
-
-  private static class AddCacheDirectiveInfoCommand implements Command {
+  private static class AddCacheDirectiveInfoCommand
+      implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-addDirective";
@@ -190,7 +124,7 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("<path>", "A path to cache. The path can be " +
           "a directory or a file.");
       listing.addRow("<pool-name>", "The pool to which the directive will be " +
@@ -252,7 +186,7 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
       }
         
-      DistributedFileSystem dfs = getDFS(conf);
+      DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       CacheDirectiveInfo directive = builder.build();
       EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
       if (force) {
@@ -262,7 +196,7 @@ public class CacheAdmin extends Configured implements Tool {
         long id = dfs.addCacheDirective(directive, flags);
         System.out.println("Added cache directive " + id);
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
 
@@ -270,7 +204,8 @@ public class CacheAdmin extends Configured implements Tool {
     }
   }
 
-  private static class RemoveCacheDirectiveInfoCommand implements Command {
+  private static class RemoveCacheDirectiveInfoCommand
+      implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-removeDirective";
@@ -283,7 +218,7 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("<id>", "The id of the cache directive to remove.  " + 
         "You must have write permission on the pool of the " +
         "directive in order to remove it.  To see a list " +
@@ -318,19 +253,20 @@ public class CacheAdmin extends Configured implements Tool {
         System.err.println("Usage is " + getShortUsage());
         return 1;
       }
-      DistributedFileSystem dfs = getDFS(conf);
+      DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         dfs.getClient().removeCacheDirective(id);
         System.out.println("Removed cached directive " + id);
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       return 0;
     }
   }
 
-  private static class ModifyCacheDirectiveInfoCommand implements Command {
+  private static class ModifyCacheDirectiveInfoCommand
+      implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-modifyDirective";
@@ -345,7 +281,7 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("<id>", "The ID of the directive to modify (required)");
       listing.addRow("<path>", "A path to cache. The path can be " +
           "a directory or a file. (optional)");
@@ -415,7 +351,7 @@ public class CacheAdmin extends Configured implements Tool {
         System.err.println("No modifications were specified.");
         return 1;
       }
-      DistributedFileSystem dfs = getDFS(conf);
+      DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       EnumSet<CacheFlag> flags = EnumSet.noneOf(CacheFlag.class);
       if (force) {
         flags.add(CacheFlag.FORCE);
@@ -424,14 +360,15 @@ public class CacheAdmin extends Configured implements Tool {
         dfs.modifyCacheDirective(builder.build(), flags);
         System.out.println("Modified cache directive " + idString);
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       return 0;
     }
   }
 
-  private static class RemoveCacheDirectiveInfosCommand implements Command {
+  private static class RemoveCacheDirectiveInfosCommand
+      implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-removeDirectives";
@@ -444,7 +381,7 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("-path <path>", "The path of the cache directives to remove.  " +
         "You must have write permission on the pool of the directive in order " +
         "to remove it.  To see a list of cache directives, use the " +
@@ -468,7 +405,7 @@ public class CacheAdmin extends Configured implements Tool {
       }
       int exitCode = 0;
       try {
-        DistributedFileSystem dfs = getDFS(conf);
+        DistributedFileSystem dfs = AdminHelper.getDFS(conf);
         RemoteIterator<CacheDirectiveEntry> iter =
             dfs.listCacheDirectives(
                 new CacheDirectiveInfo.Builder().
@@ -480,12 +417,12 @@ public class CacheAdmin extends Configured implements Tool {
             System.out.println("Removed cache directive " +
                 entry.getInfo().getId());
           } catch (IOException e) {
-            System.err.println(prettifyException(e));
+            System.err.println(AdminHelper.prettifyException(e));
             exitCode = 2;
           }
         }
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         exitCode = 2;
       }
       if (exitCode == 0) {
@@ -496,7 +433,8 @@ public class CacheAdmin extends Configured implements Tool {
     }
   }
 
-  private static class ListCacheDirectiveInfoCommand implements Command {
+  private static class ListCacheDirectiveInfoCommand
+      implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-listDirectives";
@@ -510,7 +448,7 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("-stats", "List path-based cache directive statistics.");
       listing.addRow("<path>", "List only " +
           "cache directives with this path. " +
@@ -559,7 +497,7 @@ public class CacheAdmin extends Configured implements Tool {
       }
       TableListing tableListing = tableBuilder.build();
       try {
-        DistributedFileSystem dfs = getDFS(conf);
+        DistributedFileSystem dfs = AdminHelper.getDFS(conf);
         RemoteIterator<CacheDirectiveEntry> iter =
             dfs.listCacheDirectives(builder.build());
         int numEntries = 0;
@@ -587,7 +525,7 @@ public class CacheAdmin extends Configured implements Tool {
             row.add("" + stats.getFilesNeeded());
             row.add("" + stats.getFilesCached());
           }
-          tableListing.addRow(row.toArray(new String[0]));
+          tableListing.addRow(row.toArray(new String[row.size()]));
           numEntries++;
         }
         System.out.print(String.format("Found %d entr%s%n",
@@ -596,14 +534,14 @@ public class CacheAdmin extends Configured implements Tool {
           System.out.print(tableListing);
         }
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       return 0;
     }
   }
 
-  private static class AddCachePoolCommand implements Command {
+  private static class AddCachePoolCommand implements AdminHelper.Command {
 
     private static final String NAME = "-addPool";
 
@@ -621,7 +559,7 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
 
       listing.addRow("<name>", "Name of the new pool.");
       listing.addRow("<owner>", "Username of the owner of the pool. " +
@@ -669,13 +607,13 @@ public class CacheAdmin extends Configured implements Tool {
         info.setMode(new FsPermission(mode));
       }
       String limitString = StringUtils.popOptionWithArgument("-limit", args);
-      Long limit = parseLimitString(limitString);
+      Long limit = AdminHelper.parseLimitString(limitString);
       if (limit != null) {
         info.setLimit(limit);
       }
       String maxTtlString = StringUtils.popOptionWithArgument("-maxTtl", args);
       try {
-        Long maxTtl = parseTtlString(maxTtlString);
+        Long maxTtl = AdminHelper.parseTtlString(maxTtlString);
         if (maxTtl != null) {
           info.setMaxRelativeExpiryMs(maxTtl);
         }
@@ -691,11 +629,11 @@ public class CacheAdmin extends Configured implements Tool {
         System.err.println("Usage is " + getShortUsage());
         return 1;
       }
-      DistributedFileSystem dfs = getDFS(conf);
+      DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         dfs.addCachePool(info);
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       System.out.println("Successfully added cache pool " + name + ".");
@@ -703,7 +641,7 @@ public class CacheAdmin extends Configured implements Tool {
     }
   }
 
-  private static class ModifyCachePoolCommand implements Command {
+  private static class ModifyCachePoolCommand implements AdminHelper.Command {
 
     @Override
     public String getName() {
@@ -719,7 +657,7 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
 
       listing.addRow("<name>", "Name of the pool to modify.");
       listing.addRow("<owner>", "Username of the owner of the pool");
@@ -733,7 +671,7 @@ public class CacheAdmin extends Configured implements Tool {
       return getShortUsage() + "\n" +
           WordUtils.wrap("Modifies the metadata of an existing cache pool. " +
           "See usage of " + AddCachePoolCommand.NAME + " for more details.",
-          MAX_LINE_WIDTH) + "\n\n" +
+          AdminHelper.MAX_LINE_WIDTH) + "\n\n" +
           listing.toString();
     }
 
@@ -745,11 +683,11 @@ public class CacheAdmin extends Configured implements Tool {
       Integer mode = (modeString == null) ?
           null : Integer.parseInt(modeString, 8);
       String limitString = StringUtils.popOptionWithArgument("-limit", args);
-      Long limit = parseLimitString(limitString);
+      Long limit = AdminHelper.parseLimitString(limitString);
       String maxTtlString = StringUtils.popOptionWithArgument("-maxTtl", args);
-      Long maxTtl = null;
+      Long maxTtl;
       try {
-        maxTtl = parseTtlString(maxTtlString);
+        maxTtl = AdminHelper.parseTtlString(maxTtlString);
       } catch (IOException e) {
         System.err.println(
             "Error while parsing maxTtl value: " + e.getMessage());
@@ -794,11 +732,11 @@ public class CacheAdmin extends Configured implements Tool {
             "change in the cache pool.");
         return 1;
       }
-      DistributedFileSystem dfs = getDFS(conf);
+      DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         dfs.modifyCachePool(info);
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       System.out.print("Successfully modified cache pool " + name);
@@ -827,7 +765,7 @@ public class CacheAdmin extends Configured implements Tool {
     }
   }
 
-  private static class RemoveCachePoolCommand implements Command {
+  private static class RemoveCachePoolCommand implements AdminHelper.Command {
 
     @Override
     public String getName() {
@@ -843,7 +781,7 @@ public class CacheAdmin extends Configured implements Tool {
     public String getLongUsage() {
       return getShortUsage() + "\n" +
           WordUtils.wrap("Remove a cache pool. This also uncaches paths " +
-              "associated with the pool.\n\n", MAX_LINE_WIDTH) +
+              "associated with the pool.\n\n", AdminHelper.MAX_LINE_WIDTH) +
           "<name>  Name of the cache pool to remove.\n";
     }
 
@@ -861,11 +799,11 @@ public class CacheAdmin extends Configured implements Tool {
         System.err.println("Usage is " + getShortUsage());
         return 1;
       }
-      DistributedFileSystem dfs = getDFS(conf);
+      DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         dfs.removeCachePool(name);
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       System.out.println("Successfully removed cache pool " + name + ".");
@@ -873,7 +811,7 @@ public class CacheAdmin extends Configured implements Tool {
     }
   }
 
-  private static class ListCachePoolsCommand implements Command {
+  private static class ListCachePoolsCommand implements AdminHelper.Command {
 
     @Override
     public String getName() {
@@ -887,15 +825,14 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("-stats", "Display additional cache pool statistics.");
       listing.addRow("<name>", "If specified, list only the named cache pool.");
 
       return getShortUsage() + "\n" +
           WordUtils.wrap("Display information about one or more cache pools, " +
-              "e.g. name, owner, group, permissions, etc.", MAX_LINE_WIDTH) +
-          "\n\n" +
-          listing.toString();
+              "e.g. name, owner, group, permissions, etc.",
+              AdminHelper.MAX_LINE_WIDTH) + "\n\n" + listing.toString();
     }
 
     @Override
@@ -908,7 +845,7 @@ public class CacheAdmin extends Configured implements Tool {
         System.err.println("Usage is " + getShortUsage());
         return 1;
       }
-      DistributedFileSystem dfs = getDFS(conf);
+      DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       TableListing.Builder builder = new TableListing.Builder().
           addField("NAME", Justification.LEFT).
           addField("OWNER", Justification.LEFT).
@@ -949,7 +886,7 @@ public class CacheAdmin extends Configured implements Tool {
             String maxTtlString = null;
 
             if (maxTtl != null) {
-              if (maxTtl.longValue() == CachePoolInfo.RELATIVE_EXPIRY_NEVER) {
+              if (maxTtl == CachePoolInfo.RELATIVE_EXPIRY_NEVER) {
                 maxTtlString  = "never";
               } else {
                 maxTtlString = DFSUtil.durationToString(maxTtl);
@@ -964,7 +901,7 @@ public class CacheAdmin extends Configured implements Tool {
               row.add(Long.toString(stats.getFilesNeeded()));
               row.add(Long.toString(stats.getFilesCached()));
             }
-            listing.addRow(row.toArray(new String[] {}));
+            listing.addRow(row.toArray(new String[row.size()]));
             ++numResults;
             if (name != null) {
               break;
@@ -972,7 +909,7 @@ public class CacheAdmin extends Configured implements Tool {
           }
         }
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       System.out.print(String.format("Found %d result%s.%n", numResults,
@@ -985,61 +922,7 @@ public class CacheAdmin extends Configured implements Tool {
     }
   }
 
-  private static class HelpCommand implements Command {
-    @Override
-    public String getName() {
-      return "-help";
-    }
-
-    @Override
-    public String getShortUsage() {
-      return "[-help <command-name>]\n";
-    }
-
-    @Override
-    public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
-      listing.addRow("<command-name>", "The command for which to get " +
-          "detailed help. If no command is specified, print detailed help for " +
-          "all commands");
-      return getShortUsage() + "\n" +
-        "Get detailed help about a command.\n\n" +
-        listing.toString();
-    }
-
-    @Override
-    public int run(Configuration conf, List<String> args) throws IOException {
-      if (args.size() == 0) {
-        for (Command command : COMMANDS) {
-          System.err.println(command.getLongUsage());
-        }
-        return 0;
-      }
-      if (args.size() != 1) {
-        System.out.println("You must give exactly one argument to -help.");
-        return 0;
-      }
-      String commandName = args.get(0);
-      // prepend a dash to match against the command names
-      Command command = determineCommand("-"+commandName);
-      if (command == null) {
-        System.err.print("Sorry, I don't know the command '" +
-          commandName + "'.\n");
-        System.err.print("Valid help command names are:\n");
-        String separator = "";
-        for (Command c : COMMANDS) {
-          System.err.print(separator + c.getName().substring(1));
-          separator = ", ";
-        }
-        System.err.print("\n");
-        return 1;
-      }
-      System.err.print(command.getLongUsage());
-      return 0;
-    }
-  }
-
-  private static final Command[] COMMANDS = {
+  private static final AdminHelper.Command[] COMMANDS = {
     new AddCacheDirectiveInfoCommand(),
     new ModifyCacheDirectiveInfoCommand(),
     new ListCacheDirectiveInfoCommand(),
@@ -1048,29 +931,6 @@ public class CacheAdmin extends Configured implements Tool {
     new AddCachePoolCommand(),
     new ModifyCachePoolCommand(),
     new RemoveCachePoolCommand(),
-    new ListCachePoolsCommand(),
-    new HelpCommand(),
+    new ListCachePoolsCommand()
   };
-
-  private static void printUsage(boolean longUsage) {
-    System.err.println(
-        "Usage: bin/hdfs cacheadmin [COMMAND]");
-    for (Command command : COMMANDS) {
-      if (longUsage) {
-        System.err.print(command.getLongUsage());
-      } else {
-        System.err.print("          " + command.getShortUsage());
-      }
-    }
-    System.err.println();
-  }
-
-  private static Command determineCommand(String commandName) {
-    for (int i = 0; i < COMMANDS.length; i++) {
-      if (COMMANDS[i].getName().equals(commandName)) {
-        return COMMANDS[i];
-      }
-    }
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f3a63a4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
index bb52ddd..09825b0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
@@ -24,7 +24,6 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -39,11 +38,6 @@ import org.apache.hadoop.util.Tool;
 @InterfaceAudience.Private
 public class CryptoAdmin extends Configured implements Tool {
 
-  /**
-   * Maximum length for printed lines
-   */
-  private static final int MAX_LINE_WIDTH = 80;
-
   public CryptoAdmin() {
     this(null);
   }
@@ -55,16 +49,17 @@ public class CryptoAdmin extends Configured implements Tool {
   @Override
   public int run(String[] args) throws IOException {
     if (args.length == 0) {
-      printUsage(false);
+      AdminHelper.printUsage(false, "crypto", COMMANDS);
       return 1;
     }
-    final Command command = determineCommand(args[0]);
+    final AdminHelper.Command command = AdminHelper.determineCommand(args[0],
+        COMMANDS);
     if (command == null) {
       System.err.println("Can't understand command '" + args[0] + "'");
       if (!args[0].startsWith("-")) {
         System.err.println("Command names must start with dashes.");
       }
-      printUsage(false);
+      AdminHelper.printUsage(false, "crypto", COMMANDS);
       return 1;
     }
     final List<String> argsList = new LinkedList<String>();
@@ -84,16 +79,6 @@ public class CryptoAdmin extends Configured implements Tool {
     System.exit(cryptoAdmin.run(argsArray));
   }
 
-  private static DistributedFileSystem getDFS(Configuration conf)
-      throws IOException {
-    final FileSystem fs = FileSystem.get(conf);
-    if (!(fs instanceof DistributedFileSystem)) {
-      throw new IllegalArgumentException("FileSystem " + fs.getUri() +
-      " is not an HDFS file system");
-    }
-    return (DistributedFileSystem) fs;
-  }
-
   /**
    * NN exceptions contain the stack trace as part of the exception message.
    * When it's a known error, pretty-print the error and squish the stack trace.
@@ -103,21 +88,7 @@ public class CryptoAdmin extends Configured implements Tool {
       e.getLocalizedMessage().split("\n")[0];
   }
 
-  private static TableListing getOptionDescriptionListing() {
-    final TableListing listing = new TableListing.Builder()
-      .addField("").addField("", true)
-      .wrapWidth(MAX_LINE_WIDTH).hideHeaders().build();
-    return listing;
-  }
-
-  interface Command {
-    String getName();
-    String getShortUsage();
-    String getLongUsage();
-    int run(Configuration conf, List<String> args) throws IOException;
-  }
-
-  private static class CreateZoneCommand implements Command {
+  private static class CreateZoneCommand implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-createZone";
@@ -130,7 +101,7 @@ public class CryptoAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      final TableListing listing = getOptionDescriptionListing();
+      final TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("<path>", "The path of the encryption zone to create. " +
         "It must be an empty directory.");
       listing.addRow("<keyName>", "Name of the key to use for the " +
@@ -160,7 +131,7 @@ public class CryptoAdmin extends Configured implements Tool {
         return 1;
       }
 
-      final DistributedFileSystem dfs = getDFS(conf);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         dfs.createEncryptionZone(new Path(path), keyName);
         System.out.println("Added encryption zone " + path);
@@ -173,7 +144,7 @@ public class CryptoAdmin extends Configured implements Tool {
     }
   }
 
-  private static class ListZonesCommand implements Command {
+  private static class ListZonesCommand implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-listZones";
@@ -197,11 +168,11 @@ public class CryptoAdmin extends Configured implements Tool {
         return 1;
       }
 
-      final DistributedFileSystem dfs = getDFS(conf);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         final TableListing listing = new TableListing.Builder()
           .addField("").addField("", true)
-          .wrapWidth(MAX_LINE_WIDTH).hideHeaders().build();
+          .wrapWidth(AdminHelper.MAX_LINE_WIDTH).hideHeaders().build();
         final RemoteIterator<EncryptionZone> it = dfs.listEncryptionZones();
         while (it.hasNext()) {
           EncryptionZone ez = it.next();
@@ -217,85 +188,8 @@ public class CryptoAdmin extends Configured implements Tool {
     }
   }
 
-  private static class HelpCommand implements Command {
-    @Override
-    public String getName() {
-      return "-help";
-    }
-
-    @Override
-    public String getShortUsage() {
-      return "[-help <command-name>]\n";
-    }
-
-    @Override
-    public String getLongUsage() {
-      final TableListing listing = getOptionDescriptionListing();
-      listing.addRow("<command-name>", "The command for which to get " +
-          "detailed help. If no command is specified, print detailed help for " +
-          "all commands");
-      return getShortUsage() + "\n" +
-        "Get detailed help about a command.\n\n" +
-        listing.toString();
-    }
-
-    @Override
-    public int run(Configuration conf, List<String> args) throws IOException {
-      if (args.size() == 0) {
-        for (Command command : COMMANDS) {
-          System.err.println(command.getLongUsage());
-        }
-        return 0;
-      }
-      if (args.size() != 1) {
-        System.out.println("You must give exactly one argument to -help.");
-        return 0;
-      }
-      final String commandName = args.get(0);
-      // prepend a dash to match against the command names
-      final Command command = determineCommand("-"+commandName);
-      if (command == null) {
-        System.err.print("Sorry, I don't know the command '" +
-          commandName + "'.\n");
-        System.err.print("Valid help command names are:\n");
-        String separator = "";
-        for (Command c : COMMANDS) {
-          System.err.print(separator + c.getName().substring(1));
-          separator = ", ";
-        }
-        System.err.print("\n");
-        return 1;
-      }
-      System.err.print(command.getLongUsage());
-      return 0;
-    }
-  }
-
-  private static final Command[] COMMANDS = {
+  private static final AdminHelper.Command[] COMMANDS = {
     new CreateZoneCommand(),
-    new ListZonesCommand(),
-    new HelpCommand(),
+    new ListZonesCommand()
   };
-
-  private static void printUsage(boolean longUsage) {
-    System.err.println(
-        "Usage: bin/hdfs crypto [COMMAND]");
-    for (Command command : COMMANDS) {
-      if (longUsage) {
-        System.err.print(command.getLongUsage());
-      } else {
-        System.err.print("          " + command.getShortUsage());
-      }
-    }
-    System.err.println();
-  }
-
-  private static Command determineCommand(String commandName) {
-    for (int i = 0; i < COMMANDS.length; i++) {
-      if (COMMANDS[i].getName().equals(commandName)) {
-        return COMMANDS[i];
-      }
-    }
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f3a63a4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
index 6023e40..d1b6017 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.tools;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -38,7 +37,6 @@ import java.util.List;
  * This class implements block storage policy operations.
  */
 public class StoragePolicyAdmin extends Configured implements Tool {
-  private static final int MAX_LINE_WIDTH = 80;
 
   public static void main(String[] argsArray) throws Exception {
     final StoragePolicyAdmin admin = new StoragePolicyAdmin(new
@@ -46,31 +44,6 @@ public class StoragePolicyAdmin extends Configured implements Tool {
     System.exit(admin.run(argsArray));
   }
 
-  private static DistributedFileSystem getDFS(Configuration conf)
-      throws IOException {
-    final FileSystem fs = FileSystem.get(conf);
-    if (!(fs instanceof DistributedFileSystem)) {
-      throw new IllegalArgumentException("FileSystem " + fs.getUri() +
-          " is not an HDFS file system");
-    }
-    return (DistributedFileSystem) fs;
-  }
-
-  /**
-   * NN exceptions contain the stack trace as part of the exception message.
-   * When it's a known error, pretty-print the error and squish the stack trace.
-   */
-  private static String prettifyException(Exception e) {
-    return e.getClass().getSimpleName() + ": " +
-        e.getLocalizedMessage().split("\n")[0];
-  }
-
-  private static TableListing getOptionDescriptionListing() {
-    return new TableListing.Builder()
-        .addField("").addField("", true)
-        .wrapWidth(MAX_LINE_WIDTH).hideHeaders().build();
-  }
-
   public StoragePolicyAdmin(Configuration conf) {
     super(conf);
   }
@@ -78,37 +51,32 @@ public class StoragePolicyAdmin extends Configured implements Tool {
   @Override
   public int run(String[] args) throws Exception {
     if (args.length == 0) {
-      printUsage(false);
+      AdminHelper.printUsage(false, "storagepolicies", COMMANDS);
       return 1;
     }
-    final Command command = determineCommand(args[0]);
+    final AdminHelper.Command command = AdminHelper.determineCommand(args[0],
+        COMMANDS);
     if (command == null) {
       System.err.println("Can't understand command '" + args[0] + "'");
       if (!args[0].startsWith("-")) {
         System.err.println("Command names must start with dashes.");
       }
-      printUsage(false);
+      AdminHelper.printUsage(false, "storagepolicies", COMMANDS);
       return 1;
     }
-    final List<String> argsList = new LinkedList<String>();
+    final List<String> argsList = new LinkedList<>();
     argsList.addAll(Arrays.asList(args).subList(1, args.length));
     try {
       return command.run(getConf(), argsList);
     } catch (IllegalArgumentException e) {
-      System.err.println(prettifyException(e));
+      System.err.println(AdminHelper.prettifyException(e));
       return -1;
     }
   }
 
-  interface Command {
-    String getName();
-    String getShortUsage();
-    String getLongUsage();
-    int run(Configuration conf, List<String> args) throws IOException;
-  }
-
   /** Command to list all the existing storage policies */
-  private static class ListStoragePoliciesCommand implements Command {
+  private static class ListStoragePoliciesCommand
+      implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-listPolicies";
@@ -127,7 +95,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
 
     @Override
     public int run(Configuration conf, List<String> args) throws IOException {
-      final DistributedFileSystem dfs = getDFS(conf);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         BlockStoragePolicy[] policies = dfs.getStoragePolicies();
         System.out.println("Block Storage Policies:");
@@ -137,7 +105,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
           }
         }
       } catch (IOException e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       return 0;
@@ -145,7 +113,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
   }
 
   /** Command to get the storage policy of a file/directory */
-  private static class GetStoragePolicyCommand implements Command {
+  private static class GetStoragePolicyCommand implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-getStoragePolicy";
@@ -158,7 +126,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      final TableListing listing = getOptionDescriptionListing();
+      final TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("<path>",
           "The path of the file/directory for getting the storage policy");
       return getShortUsage() + "\n" +
@@ -175,7 +143,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
         return 1;
       }
 
-      final DistributedFileSystem dfs = getDFS(conf);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         HdfsFileStatus status = dfs.getClient().getFileInfo(path);
         if (status == null) {
@@ -195,7 +163,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
           }
         }
       } catch (Exception e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       System.err.println("Cannot identify the storage policy for " + path);
@@ -204,7 +172,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
   }
 
   /** Command to set the storage policy to a file/directory */
-  private static class SetStoragePolicyCommand implements Command {
+  private static class SetStoragePolicyCommand implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-setStoragePolicy";
@@ -217,7 +185,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
 
     @Override
     public String getLongUsage() {
-      TableListing listing = getOptionDescriptionListing();
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("<path>", "The path of the file/directory to set storage" +
           " policy");
       listing.addRow("<policy>", "The name of the block storage policy");
@@ -243,97 +211,21 @@ public class StoragePolicyAdmin extends Configured implements Tool {
         return 1;
       }
 
-      final DistributedFileSystem dfs = getDFS(conf);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         dfs.setStoragePolicy(new Path(path), policyName);
         System.out.println("Set storage policy " + policyName + " on " + path);
       } catch (Exception e) {
-        System.err.println(prettifyException(e));
+        System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       return 0;
     }
   }
 
-  private static class HelpCommand implements Command {
-    @Override
-    public String getName() {
-      return "-help";
-    }
-
-    @Override
-    public String getShortUsage() {
-      return "[-help <command-name>]\n";
-    }
-
-    @Override
-    public String getLongUsage() {
-      final TableListing listing = getOptionDescriptionListing();
-      listing.addRow("<command-name>", "The command for which to get " +
-          "detailed help. If no command is specified, print detailed help for " +
-          "all commands");
-      return getShortUsage() + "\n" +
-          "Get detailed help about a command.\n\n" +
-          listing.toString();
-    }
-
-    @Override
-    public int run(Configuration conf, List<String> args) throws IOException {
-      if (args.size() == 0) {
-        for (Command command : COMMANDS) {
-          System.err.println(command.getLongUsage());
-        }
-        return 0;
-      }
-      if (args.size() != 1) {
-        System.out.println("You must give exactly one argument to -help.");
-        return 0;
-      }
-      final String commandName = args.get(0);
-      // prepend a dash to match against the command names
-      final Command command = determineCommand("-" + commandName);
-      if (command == null) {
-        System.err.print("Unknown command '" + commandName + "'.\n");
-        System.err.print("Valid help command names are:\n");
-        String separator = "";
-        for (Command c : COMMANDS) {
-          System.err.print(separator + c.getName().substring(1));
-          separator = ", ";
-        }
-        System.err.print("\n");
-        return 1;
-      }
-      System.err.print(command.getLongUsage());
-      return 0;
-    }
-  }
-
-  private static final Command[] COMMANDS = {
+  private static final AdminHelper.Command[] COMMANDS = {
       new ListStoragePoliciesCommand(),
       new SetStoragePolicyCommand(),
-      new GetStoragePolicyCommand(),
-      new HelpCommand()
+      new GetStoragePolicyCommand()
   };
-
-  private static void printUsage(boolean longUsage) {
-    System.err.println(
-        "Usage: bin/hdfs storagepolicies [COMMAND]");
-    for (Command command : COMMANDS) {
-      if (longUsage) {
-        System.err.print(command.getLongUsage());
-      } else {
-        System.err.print("          " + command.getShortUsage());
-      }
-    }
-    System.err.println();
-  }
-
-  private static Command determineCommand(String commandName) {
-    for (Command COMMAND : COMMANDS) {
-      if (COMMAND.getName().equals(commandName)) {
-        return COMMAND;
-      }
-    }
-    return null;
-  }
 }
\ No newline at end of file