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 xg...@apache.org on 2017/02/24 23:56:43 UTC

[16/26] hadoop git commit: HDFS-11426. Refactor EC CLI to be similar to storage policies CLI.

HDFS-11426. Refactor EC CLI to be similar to storage policies CLI.


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

Branch: refs/heads/YARN-5734
Commit: 132f758e3dbe3a3f11c0d9b2de8edbee594fb475
Parents: 694e680
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Feb 23 16:00:11 2017 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Feb 23 16:00:11 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/cli/CLITestHelper.java    |  15 +-
 .../hadoop-hdfs/src/main/bin/hdfs               |   2 +-
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   | 320 +++++++++++++++++++
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   |   2 +-
 .../hadoop/hdfs/tools/erasurecode/ECCli.java    |  62 ----
 .../hdfs/tools/erasurecode/ECCommand.java       | 248 --------------
 .../src/site/markdown/HDFSErasureCoding.md      |  16 +-
 .../hadoop/cli/CLITestCmdErasureCoding.java     |   4 +-
 .../cli/util/ErasureCodingCliCmdExecutor.java   |   6 +-
 .../test/resources/testErasureCodingConf.xml    | 135 ++++----
 10 files changed, 405 insertions(+), 405 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
index b08af16..89d4e30 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.cli;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.cli.util.*;
 import org.apache.hadoop.cli.util.CommandExecutor.Result;
 import org.apache.hadoop.conf.Configuration;
@@ -28,6 +26,9 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.xml.sax.Attributes;
 import org.xml.sax.SAXException;
 import org.xml.sax.helpers.DefaultHandler;
@@ -41,9 +42,9 @@ import java.util.ArrayList;
  * Tests for the Command Line Interface (CLI)
  */
 public class CLITestHelper {
-  private static final Log LOG =
-    LogFactory.getLog(CLITestHelper.class.getName());
-  
+  private static final Logger LOG = LoggerFactory.getLogger(CLITestHelper
+      .class);
+
   // In this mode, it runs the command and compares the actual output
   // with the expected output  
   public static final String TESTMODE_TEST = "test"; // Run the tests
@@ -62,7 +63,6 @@ public class CLITestHelper {
   // Storage for tests read in from the config file
   protected ArrayList<CLITestData> testsFromConfigFile = null;
   protected ArrayList<ComparatorData> testComparators = null;
-  protected String thisTestCaseName = null;
   protected ComparatorData comparatorData = null;
   protected Configuration conf = null;
   protected String clitestDataDir = null;
@@ -80,7 +80,8 @@ public class CLITestHelper {
         p.parse(testConfigFile, getConfigParser());
         success = true;
       } catch (Exception e) {
-        LOG.info("File: " + testConfigFile + " not found");
+        LOG.info("Exception while reading test config file {}:",
+            testConfigFile, e);
         success = false;
       }
       assertTrue("Error reading test config file", success);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 617adbe..cf6d94a 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -130,7 +130,7 @@ function hdfscmd_case
       exit 0
     ;;
     ec)
-      HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.erasurecode.ECCli
+      HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.ECAdmin
     ;;
     fetchdt)
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.DelegationTokenFetcher

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
new file mode 100644
index 0000000..29c65b1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
@@ -0,0 +1,320 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.tools.TableListing;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * CLI for the erasure code encoding operations.
+ */
+@InterfaceAudience.Private
+public class ECAdmin extends Configured implements Tool {
+
+  public static final String NAME = "ec";
+
+  public static void main(String[] args) throws Exception {
+    final ECAdmin admin = new ECAdmin(new Configuration());
+    int res = ToolRunner.run(admin, args);
+    System.exit(res);
+  }
+
+  public ECAdmin(Configuration conf) {
+    super(conf);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    if (args.length == 0) {
+      AdminHelper.printUsage(false, NAME, COMMANDS);
+      ToolRunner.printGenericCommandUsage(System.err);
+      return 1;
+    }
+    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.");
+      }
+      AdminHelper.printUsage(false, NAME, COMMANDS);
+      ToolRunner.printGenericCommandUsage(System.err);
+      return 1;
+    }
+    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(AdminHelper.prettifyException(e));
+      return -1;
+    }
+  }
+
+  /** Command to list the set of available erasure coding policies */
+  private static class ListECPoliciesCommand
+      implements AdminHelper.Command {
+    @Override
+    public String getName() {
+      return "-listPolicies";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + "]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      return getShortUsage() + "\n" +
+          "Get the list of supported erasure coding policies.\n";
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      if (args.size() > 0) {
+        System.err.println(getName() + ": Too many arguments");
+        return 1;
+      }
+
+      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
+      try {
+        Collection<ErasureCodingPolicy> policies =
+            dfs.getAllErasureCodingPolicies();
+        System.out.println("Erasure Coding Policies:");
+        for (ErasureCodingPolicy policy : policies) {
+          if (policy != null) {
+            System.out.println("\t" + policy.getName());
+          }
+        }
+      } catch (IOException e) {
+        System.err.println(AdminHelper.prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
+  /** Command to get the erasure coding policy for a file or directory */
+  private static class GetECPolicyCommand implements AdminHelper.Command {
+    @Override
+    public String getName() {
+      return "-getPolicy";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " -path <path>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      final TableListing listing = AdminHelper.getOptionDescriptionListing();
+      listing.addRow("<path>",
+          "The path of the file/directory for getting the erasure coding " +
+              "policy");
+      return getShortUsage() + "\n" +
+          "Get the erasure coding policy of a file/directory.\n\n" +
+          listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      final String path = StringUtils.popOptionWithArgument("-path", args);
+      if (path == null) {
+        System.err.println("Please specify the path with -path.\nUsage: " +
+            getLongUsage());
+        return 1;
+      }
+
+      if (args.size() > 0) {
+        System.err.println(getName() + ": Too many arguments");
+        return 1;
+      }
+
+      final Path p = new Path(path);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(p.toUri(), conf);
+      try {
+        ErasureCodingPolicy ecPolicy = dfs.getErasureCodingPolicy(p);
+        if (ecPolicy != null) {
+          System.out.println(ecPolicy.getName());
+        } else {
+          System.out.println("The erasure coding policy of " + path + " is " +
+              "unspecified");
+        }
+      } catch (Exception e) {
+        System.err.println(AdminHelper.prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
+  /** Command to set the erasure coding policy to a file/directory */
+  private static class SetECPolicyCommand implements AdminHelper.Command {
+    @Override
+    public String getName() {
+      return "-setPolicy";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " -path <path> -policy <policy>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
+      listing.addRow("<path>", "The path of the file/directory to set " +
+          "the erasure coding policy");
+      listing.addRow("<policy>", "The name of the erasure coding policy");
+      return getShortUsage() + "\n" +
+          "Set the erasure coding policy for a file/directory.\n\n" +
+          listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      final String path = StringUtils.popOptionWithArgument("-path", args);
+      if (path == null) {
+        System.err.println("Please specify the path for setting the EC " +
+            "policy.\nUsage: " + getLongUsage());
+        return 1;
+      }
+
+      final String ecPolicyName = StringUtils.popOptionWithArgument("-policy",
+          args);
+      if (ecPolicyName == null) {
+        System.err.println("Please specify the policy name.\nUsage: " +
+            getLongUsage());
+        return 1;
+      }
+
+      if (args.size() > 0) {
+        System.err.println(getName() + ": Too many arguments");
+        return 1;
+      }
+
+      final Path p = new Path(path);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(p.toUri(), conf);
+      try {
+        ErasureCodingPolicy ecPolicy = null;
+        ErasureCodingPolicy[] ecPolicies =
+            dfs.getClient().getErasureCodingPolicies();
+        for (ErasureCodingPolicy policy : ecPolicies) {
+          if (ecPolicyName.equals(policy.getName())) {
+            ecPolicy = policy;
+            break;
+          }
+        }
+        if (ecPolicy == null) {
+          StringBuilder sb = new StringBuilder();
+          sb.append("Policy '");
+          sb.append(ecPolicyName);
+          sb.append("' does not match any of the supported policies.");
+          sb.append(" Please select any one of ");
+          List<String> ecPolicyNames = new ArrayList<String>();
+          for (ErasureCodingPolicy policy : ecPolicies) {
+            ecPolicyNames.add(policy.getName());
+          }
+          sb.append(ecPolicyNames);
+          System.err.println(sb.toString());
+          return 3;
+        }
+        dfs.setErasureCodingPolicy(p, ecPolicy);
+        System.out.println("Set erasure coding policy " + ecPolicyName +
+            " on " + path);
+      } catch (Exception e) {
+        System.err.println(AdminHelper.prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
+  /** Command to unset the erasure coding policy set for a file/directory */
+  private static class UnsetECPolicyCommand
+      implements AdminHelper.Command {
+
+    @Override
+    public String getName() {
+      return "-unsetPolicy";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " -path <path>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
+      listing.addRow("<path>", "The path of the directory "
+          + "from which the erasure coding policy will be unset.");
+      return getShortUsage() + "\n"
+          + "Unset the erasure coding policy for a directory.\n\n"
+          + listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      final String path = StringUtils.popOptionWithArgument("-path", args);
+      if (path == null) {
+        System.err.println("Please specify a path.\nUsage: " + getLongUsage());
+        return 1;
+      }
+
+      if (args.size() > 0) {
+        System.err.println(getName() + ": Too many arguments");
+        return 1;
+      }
+
+      final Path p = new Path(path);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(p.toUri(), conf);
+      try {
+        dfs.unsetErasureCodingPolicy(p);
+        System.out.println("Unset erasure coding policy from " + path);
+      } catch (Exception e) {
+        System.err.println(AdminHelper.prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
+  private static final AdminHelper.Command[] COMMANDS = {
+      new ListECPoliciesCommand(),
+      new GetECPolicyCommand(),
+      new SetECPolicyCommand(),
+      new UnsetECPolicyCommand()
+  };
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/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 f0643b2..9c7d048 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
@@ -143,7 +143,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
     public int run(Configuration conf, List<String> args) throws IOException {
       final String path = StringUtils.popOptionWithArgument("-path", args);
       if (path == null) {
-        System.err.println("Please specify the path with -path.\nUsage:" +
+        System.err.println("Please specify the path with -path.\nUsage: " +
             getLongUsage());
         return 1;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
deleted file mode 100644
index 89dd4ee..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * 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.erasurecode;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.fs.shell.CommandFactory;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.util.ToolRunner;
-
-import java.io.IOException;
-
-/**
- * CLI for the erasure code encoding operations.
- */
-@InterfaceAudience.Private
-public class ECCli extends FsShell {
-
-  private final static String usagePrefix =
-      "Usage: hdfs ec [generic options]";
-
-  @Override
-  protected String getUsagePrefix() {
-    return usagePrefix;
-  }
-
-  @Override
-  protected void init() throws IOException {
-    getConf().setQuietMode(true);
-    if (commandFactory == null) {
-      commandFactory = new CommandFactory(getConf());
-      commandFactory.addObject(getHelp(), "-help");
-      registerCommands(commandFactory);
-    }
-  }
-
-  @Override
-  protected void registerCommands(CommandFactory factory) {
-    factory.registerCommands(ECCommand.class);
-  }
-
-  public static void main(String[] args) throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    int res = ToolRunner.run(conf, new ECCli(), args);
-    System.exit(res);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
deleted file mode 100644
index fc732e0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/**
- * 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.erasurecode;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.shell.Command;
-import org.apache.hadoop.fs.shell.CommandFactory;
-import org.apache.hadoop.fs.shell.PathData;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.util.StringUtils;
-
-/**
- * Erasure Coding CLI commands
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public abstract class ECCommand extends Command {
-
-  public static void registerCommands(CommandFactory factory) {
-    // Register all commands of Erasure CLI, with a '-' at the beginning in name
-    // of the command.
-    factory.addClass(SetECPolicyCommand.class, "-" + SetECPolicyCommand.NAME);
-    factory.addClass(GetECPolicyCommand.class, "-"
-        + GetECPolicyCommand.NAME);
-    factory.addClass(UnsetECPolicyCommand.class, "-"
-        + UnsetECPolicyCommand.NAME);
-    factory.addClass(ListPolicies.class, "-" + ListPolicies.NAME);
-  }
-
-  @Override
-  public String getCommandName() {
-    return getName();
-  }
-
-  @Override
-  protected void run(Path path) throws IOException {
-    throw new RuntimeException("Not suppose to get here");
-  }
-
-  @Deprecated
-  @Override
-  public int runAll() {
-    return run(args);
-  }
-
-  @Override
-  protected void processPath(PathData item) throws IOException {
-    if (!(item.fs instanceof DistributedFileSystem)) {
-      throw new UnsupportedActionException(
-          "Erasure commands are only supported for the HDFS paths");
-    }
-  }
-
-  /**
-   * A command to set the erasure coding policy for a directory, with the name
-   * of the policy.
-   */
-  static class SetECPolicyCommand extends ECCommand {
-    public static final String NAME = "setPolicy";
-    public static final String USAGE = "[-p <policyName>] <path>";
-    public static final String DESCRIPTION = 
-        "Set a specified erasure coding policy to a directory\n"
-        + "Options :\n"
-        + "  -p <policyName> : erasure coding policy name to encode files. "
-        + "If not passed the default policy will be used\n"
-        + "  <path>  : Path to a directory. Under this directory "
-        + "files will be encoded using specified erasure coding policy";
-    private String ecPolicyName;
-    private ErasureCodingPolicy ecPolicy = null;
-
-    @Override
-    protected void processOptions(LinkedList<String> args) throws IOException {
-      ecPolicyName = StringUtils.popOptionWithArgument("-p", args);
-      if (args.isEmpty()) {
-        throw new HadoopIllegalArgumentException("<path> is missing");
-      }
-      if (args.size() > 1) {
-        throw new HadoopIllegalArgumentException("Too many arguments");
-      }
-    }
-
-    @Override
-    protected void processPath(PathData item) throws IOException {
-      super.processPath(item);
-      DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
-      try {
-        if (ecPolicyName != null) {
-          ErasureCodingPolicy[] ecPolicies = dfs.getClient().getErasureCodingPolicies();
-          for (ErasureCodingPolicy ecPolicy : ecPolicies) {
-            if (ecPolicyName.equals(ecPolicy.getName())) {
-              this.ecPolicy = ecPolicy;
-              break;
-            }
-          }
-          if (ecPolicy == null) {
-            StringBuilder sb = new StringBuilder();
-            sb.append("Policy '");
-            sb.append(ecPolicyName);
-            sb.append("' does not match any of the supported policies.");
-            sb.append(" Please select any one of ");
-            List<String> ecPolicyNames = new ArrayList<String>();
-            for (ErasureCodingPolicy ecPolicy : ecPolicies) {
-              ecPolicyNames.add(ecPolicy.getName());
-            }
-            sb.append(ecPolicyNames);
-            throw new HadoopIllegalArgumentException(sb.toString());
-          }
-        }
-        dfs.setErasureCodingPolicy(item.path, ecPolicy);
-        out.println("EC policy set successfully at " + item.path);
-      } catch (IOException e) {
-        throw new IOException("Unable to set EC policy for the path "
-            + item.path + ". " + e.getMessage());
-      }
-    }
-  }
-
-  /**
-   * Get the erasure coding policy of a file or directory
-   */
-  static class GetECPolicyCommand extends ECCommand {
-    public static final String NAME = "getPolicy";
-    public static final String USAGE = "<path>";
-    public static final String DESCRIPTION =
-        "Get erasure coding policy information about at specified path\n";
-
-    @Override
-    protected void processOptions(LinkedList<String> args) throws IOException {
-      if (args.isEmpty()) {
-        throw new HadoopIllegalArgumentException("<path> is missing");
-      }
-      if (args.size() > 1) {
-        throw new HadoopIllegalArgumentException("Too many arguments");
-      }
-    }
-
-    @Override
-    protected void processPath(PathData item) throws IOException {
-      super.processPath(item);
-      DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
-      try {
-        ErasureCodingPolicy ecPolicy = dfs.getErasureCodingPolicy(item.path);
-        if (ecPolicy != null) {
-          out.println(ecPolicy.toString());
-        } else {
-          out.println("Path " + item.path + " is not erasure coded.");
-        }
-      } catch (IOException e) {
-        throw new IOException("Unable to get EC policy for the path "
-            + item.path + ". " + e.getMessage());
-      }
-    }
-  }
-
-  /**
-   * List all supported erasure coding policies
-   */
-  static class ListPolicies extends ECCommand {
-    public static final String NAME = "listPolicies";
-    public static final String USAGE = "";
-    public static final String DESCRIPTION = 
-        "Get the list of erasure coding policies supported\n";
-
-    @Override
-    protected void processOptions(LinkedList<String> args) throws IOException {
-      if (!args.isEmpty()) {
-        throw new HadoopIllegalArgumentException("Too many parameters");
-      }
-
-      FileSystem fs = FileSystem.get(getConf());
-      if (fs instanceof DistributedFileSystem == false) {
-        throw new UnsupportedActionException(
-            "Erasure commands are only supported for the HDFS");
-      }
-      DistributedFileSystem dfs = (DistributedFileSystem) fs;
-
-      ErasureCodingPolicy[] ecPolicies = dfs.getClient().getErasureCodingPolicies();
-      StringBuilder sb = new StringBuilder();
-      int i = 0;
-      while (i < ecPolicies.length) {
-        ErasureCodingPolicy ecPolicy = ecPolicies[i];
-        sb.append(ecPolicy.getName());
-        i++;
-        if (i < ecPolicies.length) {
-          sb.append(", ");
-        }
-      }
-      out.println(sb.toString());
-    }
-  }
-
-  /**
-   * Unset the erasure coding policy from a directory.
-   */
-  static class UnsetECPolicyCommand extends ECCommand {
-    public static final String NAME = "unsetPolicy";
-    public static final String USAGE = "<path>";
-    public static final String DESCRIPTION =
-        "Unset erasure coding policy from a directory\n";
-
-    @Override
-    protected void processOptions(LinkedList<String> args) throws IOException {
-      if (args.isEmpty()) {
-        throw new HadoopIllegalArgumentException("<path> is missing");
-      }
-      if (args.size() > 1) {
-        throw new HadoopIllegalArgumentException("Too many arguments");
-      }
-    }
-
-    @Override
-    protected void processPath(PathData item) throws IOException {
-      super.processPath(item);
-      DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
-      try {
-        dfs.unsetErasureCodingPolicy(item.path);
-      } catch (IOException e) {
-        throw new IOException("Unable to unset EC policy from directory "
-            + item.path + ". " + e.getMessage());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 6e4891f..0283e2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -119,30 +119,30 @@ Deployment
   HDFS provides an `ec` subcommand to perform administrative commands related to erasure coding.
 
        hdfs ec [generic options]
-         [-setPolicy [-p <policyName>] <path>]
-         [-getPolicy <path>]
-         [-unsetPolicy <path>]
+         [-setPolicy -policy <policyName> -path <path>]
+         [-getPolicy -path <path>]
+         [-unsetPolicy -path <path>]
          [-listPolicies]
          [-usage [cmd ...]]
          [-help [cmd ...]]
 
 Below are the details about each command.
 
- *  `[-setPolicy [-p <policyName>] <path>]`
+ *  `[-setPolicy -policy <policyName> -path <path>]`
 
     Sets an ErasureCoding policy on a directory at the specified path.
 
       `path`: An directory in HDFS. This is a mandatory parameter. Setting a policy only affects newly created files, and does not affect existing files.
 
-      `policyName`: The ErasureCoding policy to be used for files under this directory. This is an optional parameter, specified using \u2018-p\u2019 flag. If no policy is specified, the system default ErasureCodingPolicy will be used.
+      `policyName`: The ErasureCoding policy to be used for files under this directory.
 
- *  `[-getPolicy <path>]`
+ *  `[-getPolicy -path <path>]`
 
      Get details of the ErasureCoding policy of a file or directory at the specified path.
 
- *  `[-unsetPolicy <path>]`
+ *  `[-unsetPolicy -path <path>]`
 
-     Unset an ErasureCoding policy from a directory at the specified path when previously user sets the ErasureCoding policy on this directory via "setPolicy" command. If the directory inherits the ErasureCoding policy from its parent group, "unsetPolicy" command on this directory will not have any effect. Unset ErasureCoding policy on a directory which doesn't have ErasureCoding policy will not return an error.
+     Unset an ErasureCoding policy set by a previous call to "setPolicy" on a directory. If the directory inherits the ErasureCoding policy from an ancestor directory, "unsetPolicy" is a no-op. Unsetting the policy on a directory which doesn't have an explicit policy set will not return an error.
 
  *  `[-listPolicies]`
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
index 0499a2b..28e5f98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.cli.util.CLITestCmd;
 import org.apache.hadoop.cli.util.CommandExecutor;
 import org.apache.hadoop.cli.util.ErasureCodingCliCmdExecutor;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.tools.erasurecode.ECCli;
+import org.apache.hadoop.hdfs.tools.ECAdmin;
 
 public class CLITestCmdErasureCoding extends CLITestCmd {
   public CLITestCmdErasureCoding(String str, CLICommandTypes type) {
@@ -33,7 +33,7 @@ public class CLITestCmdErasureCoding extends CLITestCmd {
   @Override
   public CommandExecutor getExecutor(String tag, Configuration conf) throws IllegalArgumentException {
     if (getType() instanceof CLICommandErasureCodingCli)
-      return new ErasureCodingCliCmdExecutor(tag, new ECCli());
+      return new ErasureCodingCliCmdExecutor(tag, new ECAdmin(conf));
     return super.getExecutor(tag, conf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java
index e993313..59b2a73 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java
@@ -17,14 +17,14 @@
  */
 package org.apache.hadoop.cli.util;
 
-import org.apache.hadoop.hdfs.tools.erasurecode.ECCli;
+import org.apache.hadoop.hdfs.tools.ECAdmin;
 import org.apache.hadoop.util.ToolRunner;
 
 public class ErasureCodingCliCmdExecutor extends CommandExecutor {
   protected String namenode = null;
-  protected ECCli admin = null;
+  protected ECAdmin admin = null;
 
-  public ErasureCodingCliCmdExecutor(String namenode, ECCli admin) {
+  public ErasureCodingCliCmdExecutor(String namenode, ECAdmin admin) {
     this.namenode = namenode;
     this.admin = admin;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132f758e/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index 0753d4d..82b71def 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -42,7 +42,11 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>-usage: Unknown command</expected-output>
+          <expected-output>Can't understand command '-usage'</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Usage: bin/hdfs ec [COMMAND]</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -58,7 +62,11 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Usage: hdfs ec [generic options]</expected-output>
+          <expected-output>[-listPolicies]</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>[-unsetPolicy -path</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -72,12 +80,12 @@
       </cleanup-commands>
       <comparators>
         <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^[ \t]*Set a specified erasure coding policy to a directory( )*</expected-output>
+          <type>SubstringComparator</type>
+          <expected-output>Set the erasure coding policy for a file/directory.</expected-output>
         </comparator>
         <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^-setPolicy \[-p &lt;policyName&gt;\] &lt;path&gt;(.)*</expected-output>
+          <type>SubstringComparator</type>
+          <expected-output>[-setPolicy -path &lt;path&gt; -policy &lt;policy&gt;]</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -92,11 +100,11 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Get erasure coding policy information about at specified path</expected-output>
+          <expected-output>Get the erasure coding policy of a file/directory</expected-output>
         </comparator>
         <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^-getPolicy &lt;path&gt;(.)*</expected-output>
+          <type>SubstringComparator</type>
+          <expected-output>[-getPolicy -path &lt;path&gt;]</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -111,11 +119,11 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Get the list of erasure coding policies supported</expected-output>
+          <expected-output>Get the list of supported erasure coding policies</expected-output>
         </comparator>
         <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^-listPolicies (.)*</expected-output>
+          <type>SubstringComparator</type>
+          <expected-output>[-listPolicies]</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -125,7 +133,7 @@
       <description>setPolicy : set erasure coding policy on a directory to encode files</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-DEFAULT-6-3-64k /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-DEFAULT-6-3-64k -path /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
@@ -133,7 +141,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>EC policy set successfully at NAMENODE/ecdir</expected-output>
+          <expected-output>Set erasure coding policy RS-DEFAULT-6-3-64k on /ecdir</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -142,8 +150,8 @@
       <description>setPolicy : set a policy twice</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-DEFAULT-6-3-64k -path /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-DEFAULT-6-3-64k -path /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
@@ -151,25 +159,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>EC policy set successfully at NAMENODE/ecdir</expected-output>
-        </comparator>
-      </comparators>
-    </test>
-
-    <test>
-      <description>setPolicy : default policy</description>
-      <test-commands>
-        <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
-      </test-commands>
-      <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /ecdir</command>
-      </cleanup-commands>
-      <comparators>
-        <comparator>
-          <type>SubstringComparator</type>
-          <expected-output>ErasureCodingPolicy=[Name=</expected-output>
+          <expected-output>Set erasure coding policy RS-DEFAULT-6-3-64k on /ecdir</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -178,9 +168,9 @@
       <description>unsetPolicy : unset policy and get</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -unsetPolicy /ecdir</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-DEFAULT-6-3-64k -path /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -unsetPolicy -path /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
@@ -188,7 +178,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>is not erasure coded.</expected-output>
+          <expected-output>is unspecified</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -197,10 +187,9 @@
       <description>setPolicy : change different policy and get</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-DEFAULT-3-2-64k
-          /ecdir</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-DEFAULT-6-3-64k -path /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-DEFAULT-3-2-64k -path /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
@@ -208,7 +197,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>ErasureCodingPolicy=[Name=RS-DEFAULT-3-2-64k</expected-output>
+          <expected-output>RS-DEFAULT-3-2-64k</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -218,10 +207,10 @@
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
         <command>-fs NAMENODE -mkdir /ecdir/child</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -unsetPolicy /ecdir/child</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-DEFAULT-6-3-64k -path /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -unsetPolicy -path /ecdir/child</ec-admin-command>
         <command>-fs NAMENODE -touchz /ecdir/child/ecfile</command>
-        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir/child/ecfile</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir/child/ecfile</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rm /ecdir/child/ecfile</command>
@@ -231,7 +220,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>ErasureCodingPolicy=[Name=RS-DEFAULT-6-3-64k</expected-output>
+          <expected-output>RS-DEFAULT-6-3-64k</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -240,7 +229,7 @@
       <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /noec</command>
-        <ec-admin-command>-fs NAMENODE -getPolicy /noec</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /noec</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /noec</command>
@@ -248,7 +237,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Path NAMENODE/noec is not erasure coded</expected-output>
+          <expected-output>is unspecified</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -257,8 +246,8 @@
       <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-DEFAULT-6-3-64k /ecdir</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-DEFAULT-6-3-64k -path /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
@@ -266,7 +255,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>ErasureCodingPolicy=[Name=RS-DEFAULT-6-3-64k</expected-output>
+          <expected-output>RS-DEFAULT-6-3-64k</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -275,9 +264,9 @@
       <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-DEFAULT-6-3-64k /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-DEFAULT-6-3-64k -path /ecdir</ec-admin-command>
         <command>-fs NAMENODE -touchz /ecdir/ecfile</command>
-        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir/ecfile</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir/ecfile</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rm /ecdir/ecfile</command>
@@ -286,7 +275,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>ErasureCodingPolicy=[Name=RS-DEFAULT-6-3-64k</expected-output>
+          <expected-output>RS-DEFAULT-6-3-64k</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -318,8 +307,8 @@
       </cleanup-commands>
       <comparators>
         <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^-setPolicy: &lt;path&gt; is missing(.)*</expected-output>
+          <type>SubstringComparator</type>
+          <expected-output>Please specify the path for setting the EC policy.</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -328,15 +317,15 @@
       <description>setPolicy : illegal parameters - policy name is missing</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -p</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^-setPolicy: option -p requires 1 argument(.)*</expected-output>
+          <type>SubstringComparator</type>
+          <expected-output>option -path requires 1 argument</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -345,7 +334,7 @@
       <description>setPolicy : illegal parameters - too many arguments</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir1 /ecdir2</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir1 -policy RS-DEFAULT-3-2-64k /ecdir2</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
@@ -362,7 +351,7 @@
       <description>setPolicy : illegal parameters - invalidpolicy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -p invalidpolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy invalidpolicy -path /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
@@ -378,14 +367,14 @@
     <test>
       <description>setPolicy : illegal parameters - no such file</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir -policy RS-DEFAULT-3-2-64k</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^setPolicy: `/ecdir': No such file or directory(.)*</expected-output>
+          <type>SubstringComparator</type>
+          <expected-output>Path not found: /ecdir</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -399,8 +388,8 @@
       </cleanup-commands>
       <comparators>
         <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^-getPolicy: &lt;path&gt; is missing(.)*</expected-output>
+          <type>SubstringComparator</type>
+          <expected-output>Please specify the path with -path</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -408,7 +397,7 @@
     <test>
       <description>getPolicy : illegal parameters - too many arguments</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rm /ecdir</command>
@@ -424,14 +413,14 @@
     <test>
       <description>getPolicy : illegal parameters - no such file</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^getPolicy: `/ecdir': No such file or directory(.)*</expected-output>
+          <type>SubstringComparator</type>
+          <expected-output>Path not found: /ecdir</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -446,7 +435,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>-listPolicies: Too many parameters</expected-output>
+          <expected-output>-listPolicies: Too many arguments</expected-output>
         </comparator>
       </comparators>
     </test>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org