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 st...@apache.org on 2017/06/30 14:59:57 UTC

[16/35] hadoop git commit: HDFS-11870. Add CLI cmd to enable/disable an erasure code policy. Contributed by lufei.

HDFS-11870. Add CLI cmd to enable/disable an erasure code policy. Contributed by lufei.


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

Branch: refs/heads/HADOOP-13345
Commit: f99b6d19de77c6e730fed8444f8848a7e63d6130
Parents: ee243e5
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Wed Jun 28 10:53:52 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Wed Jun 28 10:54:25 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  12 ++
 .../hadoop/hdfs/DistributedFileSystem.java      |  22 +++
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |  22 +++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  17 ++
 .../ClientNamenodeProtocolTranslatorPB.java     |  30 ++++
 .../src/main/proto/ClientNamenodeProtocol.proto |   4 +
 .../src/main/proto/erasurecoding.proto          |  14 ++
 ...tNamenodeProtocolServerSideTranslatorPB.java |  28 ++++
 .../namenode/ErasureCodingPolicyManager.java    |  56 +++++++
 .../server/namenode/FSDirErasureCodingOp.java   |  12 ++
 .../hdfs/server/namenode/FSNamesystem.java      |  56 ++++++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  16 ++
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   |  99 +++++++++++-
 .../src/site/markdown/HDFSCommands.md           |   4 +
 .../src/site/markdown/HDFSErasureCoding.md      |  12 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |  48 ++++++
 .../test/resources/testErasureCodingConf.xml    | 162 +++++++++++++++++++
 17 files changed, 611 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index d114f0f..1f6022c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -2783,6 +2783,18 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     namenode.removeErasureCodingPolicy(ecPolicyName);
   }
 
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    checkOpen();
+    namenode.enableErasureCodingPolicy(ecPolicyName);
+  }
+
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    checkOpen();
+    namenode.disableErasureCodingPolicy(ecPolicyName);
+  }
+
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
     checkOpen();
     return new DFSInotifyEventInputStream(namenode, tracer);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index f8af4ab..34c631a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2619,6 +2619,28 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   /**
+   * Enable erasure coding policy.
+   *
+   * @param ecPolicyName The name of the policy to be enabled.
+   * @throws IOException
+   */
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    dfs.enableErasureCodingPolicy(ecPolicyName);
+  }
+
+  /**
+   * Disable erasure coding policy.
+   *
+   * @param ecPolicyName The name of the policy to be disabled.
+   * @throws IOException
+   */
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    dfs.disableErasureCodingPolicy(ecPolicyName);
+  }
+
+  /**
    * Unset the erasure coding policy from the source path.
    *
    * @param path     The directory to unset the policy

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
index 701bf0f..abf341e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
@@ -548,6 +548,28 @@ public class HdfsAdmin {
     dfs.removeErasureCodingPolicy(ecPolicyName);
   }
 
+  /**
+   * Enable erasure coding policy.
+   *
+   * @param ecPolicyName The name of the policy to be enabled.
+   * @throws IOException
+   */
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    dfs.enableErasureCodingPolicy(ecPolicyName);
+  }
+
+  /**
+   * Disable erasure coding policy.
+   *
+   * @param ecPolicyName The name of the policy to be disabled.
+   * @throws IOException
+   */
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    dfs.disableErasureCodingPolicy(ecPolicyName);
+  }
+
   private void provisionEZTrash(Path path) throws IOException {
     // make sure the path is an EZ
     EncryptionZone ez = dfs.getEZForPath(path);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 313f322..45c6b32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1571,6 +1571,23 @@ public interface ClientProtocol {
   void removeErasureCodingPolicy(String ecPolicyName) throws IOException;
 
   /**
+   * Enable erasure coding policy.
+   * @param ecPolicyName The name of the policy to be enabled.
+   * @throws IOException
+   */
+  @AtMostOnce
+  void enableErasureCodingPolicy(String ecPolicyName) throws IOException;
+
+  /**
+   * Disable erasure coding policy.
+   * @param ecPolicyName The name of the policy to be disabled.
+   * @throws IOException
+   */
+  @AtMostOnce
+  void disableErasureCodingPolicy(String ecPolicyName) throws IOException;
+
+
+  /**
    * Get the erasure coding policies loaded in Namenode.
    *
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index d913f81..388788c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -186,6 +186,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodin
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.RemoveErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.EnableErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.DisableErasureCodingPolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingCodecsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingCodecsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto;
@@ -1709,6 +1711,34 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    EnableErasureCodingPolicyRequestProto.Builder builder =
+        EnableErasureCodingPolicyRequestProto.newBuilder();
+    builder.setEcPolicyName(ecPolicyName);
+    EnableErasureCodingPolicyRequestProto req = builder.build();
+    try {
+      rpcProxy.enableErasureCodingPolicy(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    DisableErasureCodingPolicyRequestProto.Builder builder =
+        DisableErasureCodingPolicyRequestProto.newBuilder();
+    builder.setEcPolicyName(ecPolicyName);
+    DisableErasureCodingPolicyRequestProto req = builder.build();
+    try {
+      rpcProxy.disableErasureCodingPolicy(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
   public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
     try {
       GetErasureCodingPoliciesResponseProto response = rpcProxy

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index d3e15d0..4f44c5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -957,6 +957,10 @@ service ClientNamenodeProtocol {
       returns(AddErasureCodingPoliciesResponseProto);
   rpc removeErasureCodingPolicy(RemoveErasureCodingPolicyRequestProto)
       returns(RemoveErasureCodingPolicyResponseProto);
+  rpc enableErasureCodingPolicy(EnableErasureCodingPolicyRequestProto)
+      returns(EnableErasureCodingPolicyResponseProto);
+  rpc disableErasureCodingPolicy(DisableErasureCodingPolicyRequestProto)
+      returns(DisableErasureCodingPolicyResponseProto);
   rpc getErasureCodingPolicy(GetErasureCodingPolicyRequestProto)
       returns(GetErasureCodingPolicyResponseProto);
   rpc getErasureCodingCodecs(GetErasureCodingCodecsRequestProto)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
index ed0fd81..65baab6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
@@ -68,6 +68,20 @@ message RemoveErasureCodingPolicyRequestProto {
 message RemoveErasureCodingPolicyResponseProto {
 }
 
+message EnableErasureCodingPolicyRequestProto {
+  required string ecPolicyName = 1;
+}
+
+message EnableErasureCodingPolicyResponseProto {
+}
+
+message DisableErasureCodingPolicyRequestProto {
+  required string ecPolicyName = 1;
+}
+
+message DisableErasureCodingPolicyResponseProto {
+}
+
 message UnsetErasureCodingPolicyRequestProto {
   required string src = 1;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 16adc70..4ac49fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -230,6 +230,10 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodin
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.RemoveErasureCodingPolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.RemoveErasureCodingPolicyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.EnableErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.EnableErasureCodingPolicyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.DisableErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.DisableErasureCodingPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCodingPolicyRequestProto;
@@ -1708,6 +1712,30 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+  public EnableErasureCodingPolicyResponseProto enableErasureCodingPolicy(
+      RpcController controller, EnableErasureCodingPolicyRequestProto request)
+      throws ServiceException {
+    try {
+      server.enableErasureCodingPolicy(request.getEcPolicyName());
+      return EnableErasureCodingPolicyResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public DisableErasureCodingPolicyResponseProto disableErasureCodingPolicy(
+      RpcController controller, DisableErasureCodingPolicyRequestProto request)
+      throws ServiceException {
+    try {
+      server.disableErasureCodingPolicy(request.getEcPolicyName());
+      return DisableErasureCodingPolicyResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public GetErasureCodingPolicyResponseProto getErasureCodingPolicy(RpcController controller,
       GetErasureCodingPolicyRequestProto request) throws ServiceException {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index d22e47e..266d45c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -212,6 +212,10 @@ public final class ErasureCodingPolicyManager {
     // This is a placeholder for HDFS-7337.
   }
 
+  /**
+   * Add an erasure coding policy.
+   * @return the added policy
+   */
   public synchronized ErasureCodingPolicy addPolicy(ErasureCodingPolicy policy)
       throws IllegalECPolicyException {
     if (!CodecUtil.hasCodec(policy.getCodecName())) {
@@ -251,6 +255,9 @@ public final class ErasureCodingPolicyManager {
     return (byte) (currentId + 1);
   }
 
+  /**
+   * Remove an User erasure coding policy by policyName.
+   */
   public synchronized void removePolicy(String name) {
     if (SystemErasureCodingPolicies.getByName(name) != null) {
       throw new IllegalArgumentException("System erasure coding policy " +
@@ -268,4 +275,53 @@ public final class ErasureCodingPolicyManager {
   public List<ErasureCodingPolicy> getRemovedPolicies() {
     return removedPoliciesByName.values().stream().collect(Collectors.toList());
   }
+
+  /**
+   * Disable an erasure coding policy by policyName.
+   */
+  public synchronized void disablePolicy(String name) {
+    ErasureCodingPolicy sysEcPolicy = SystemErasureCodingPolicies
+        .getByName(name);
+    ErasureCodingPolicy userEcPolicy = userPoliciesByName.get(name);
+    LOG.info("Disable the erasure coding policy " + name);
+    if (sysEcPolicy == null &&
+        userEcPolicy == null) {
+      throw new IllegalArgumentException("The policy name " +
+          name + " does not exists");
+    }
+
+    if(sysEcPolicy != null){
+      enabledPoliciesByName.remove(name);
+      removedPoliciesByName.put(name, sysEcPolicy);
+    }
+    if(userEcPolicy != null){
+      enabledPoliciesByName.remove(name);
+      removedPoliciesByName.put(name, userEcPolicy);
+    }
+  }
+
+  /**
+   * Enable an erasure coding policy by policyName.
+   */
+  public synchronized void enablePolicy(String name) {
+    ErasureCodingPolicy sysEcPolicy = SystemErasureCodingPolicies
+        .getByName(name);
+    ErasureCodingPolicy userEcPolicy = userPoliciesByName.get(name);
+    LOG.info("Enable the erasure coding policy " + name);
+    if (sysEcPolicy == null &&
+        userEcPolicy == null) {
+      throw new IllegalArgumentException("The policy name " +
+          name + " does not exists");
+    }
+
+    if(sysEcPolicy != null){
+      enabledPoliciesByName.put(name, sysEcPolicy);
+      removedPoliciesByName.remove(name);
+    }
+    if(userEcPolicy != null) {
+      enabledPoliciesByName.put(name, userEcPolicy);
+      removedPoliciesByName.remove(name);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index a040262..681f217 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -231,6 +231,18 @@ final class FSDirErasureCodingOp {
     fsn.getErasureCodingPolicyManager().removePolicy(ecPolicyName);
   }
 
+  static void enableErasureCodePolicy(final FSNamesystem fsn,
+      String ecPolicyName) throws IOException {
+    Preconditions.checkNotNull(ecPolicyName);
+    fsn.getErasureCodingPolicyManager().enablePolicy(ecPolicyName);
+  }
+
+  static void disableErasureCodePolicy(final FSNamesystem fsn,
+      String ecPolicyName) throws IOException {
+    Preconditions.checkNotNull(ecPolicyName);
+    fsn.getErasureCodingPolicyManager().disablePolicy(ecPolicyName);
+  }
+
   private static List<XAttr> removeErasureCodingPolicyXAttr(
       final FSNamesystem fsn, final INodesInPath srcIIP) throws IOException {
     FSDirectory fsd = fsn.getFSDirectory();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index f7acb55..9872cd7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7060,6 +7060,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   AddECPolicyResponse[] addECPolicies(ErasureCodingPolicy[] policies)
       throws IOException {
     final String operationName = "addECPolicies";
+    String addECPolicyName = "";
     checkOperation(OperationCategory.WRITE);
     List<AddECPolicyResponse> responses = new ArrayList<>();
     boolean success = false;
@@ -7070,6 +7071,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         try {
           ErasureCodingPolicy newPolicy =
               FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
+          addECPolicyName = newPolicy.getName();
           responses.add(new AddECPolicyResponse(newPolicy));
         } catch (IllegalECPolicyException e) {
           responses.add(new AddECPolicyResponse(policy, e));
@@ -7082,7 +7084,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (success) {
         getEditLog().logSync();
       }
-      logAuditEvent(success, operationName, null, null, null);
+      logAuditEvent(success, operationName, addECPolicyName, null, null);
     }
   }
 
@@ -7109,6 +7111,58 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
+   * Enable an erasure coding policy.
+   * @param ecPolicyName the name of the policy to be enabled
+   * @throws IOException
+   */
+  void enableErasureCodingPolicy(String ecPolicyName) throws IOException {
+    final String operationName = "enableErasureCodingPolicy";
+    checkOperation(OperationCategory.WRITE);
+    boolean success = false;
+    LOG.info("Enable the erasure coding policy " + ecPolicyName);
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot enable erasure coding policy "
+          + ecPolicyName);
+      FSDirErasureCodingOp.enableErasureCodePolicy(this, ecPolicyName);
+      success = true;
+    } finally {
+      writeUnlock(operationName);
+      if (success) {
+        getEditLog().logSync();
+      }
+      logAuditEvent(success, operationName, ecPolicyName, null, null);
+    }
+  }
+
+  /**
+   * Disable an erasure coding policy.
+   * @param ecPolicyName the name of the policy to be disabled
+   * @throws IOException
+   */
+  void disableErasureCodingPolicy(String ecPolicyName) throws IOException {
+    final String operationName = "disableErasureCodingPolicy";
+    checkOperation(OperationCategory.WRITE);
+    boolean success = false;
+    LOG.info("Disable the erasure coding policy " + ecPolicyName);
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot disable erasure coding policy "
+          + ecPolicyName);
+      FSDirErasureCodingOp.disableErasureCodePolicy(this, ecPolicyName);
+      success = true;
+    } finally {
+      writeUnlock(operationName);
+      if (success) {
+        getEditLog().logSync();
+      }
+      logAuditEvent(success, operationName, ecPolicyName, null, null);
+    }
+  }
+
+  /**
    * Unset an erasure coding policy from the given path.
    * @param srcArg  The path of the target directory.
    * @throws AccessControlException  if the caller is not the superuser.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index fd5a05b..39d93df 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -2307,6 +2307,22 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.removeErasureCodingPolicy(ecPolicyName);
   }
 
+  @Override // ClientProtocol
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    checkNNStartup();
+    namesystem.checkSuperuserPrivilege();
+    namesystem.enableErasureCodingPolicy(ecPolicyName);
+  }
+
+  @Override // ClientProtocol
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    checkNNStartup();
+    namesystem.checkSuperuserPrivilege();
+    namesystem.disableErasureCodingPolicy(ecPolicyName);
+  }
+
   @Override // ReconfigurationProtocol
   public void startReconfiguration() throws IOException {
     checkNNStartup();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/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
index a983056..5006b5a 100644
--- 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
@@ -463,6 +463,101 @@ public class ECAdmin extends Configured implements Tool {
     }
   }
 
+  /** Command to enable an existing erasure coding policy. */
+  private static class EnableECPolicyCommand implements AdminHelper.Command {
+    @Override
+    public String getName() {
+      return "-enablePolicy";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " -policy <policy>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
+      listing.addRow("<policy>", "The name of the erasure coding policy");
+      return getShortUsage() + "\n" +
+          "Enable the erasure coding policy.\n\n" +
+          listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      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 DistributedFileSystem dfs = AdminHelper.getDFS(conf);
+      try {
+        dfs.enableErasureCodingPolicy(ecPolicyName);
+        System.out.println("Erasure coding policy " + ecPolicyName +
+            " is enabled");
+      } catch (IOException e) {
+        System.err.println(AdminHelper.prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
+  /** Command to disable an existing erasure coding policy. */
+  private static class DisableECPolicyCommand implements AdminHelper.Command {
+    @Override
+    public String getName() {
+      return "-disablePolicy";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " -policy <policy>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
+      listing.addRow("<policy>", "The name of the erasure coding policy");
+      return getShortUsage() + "\n" +
+          "Disable the erasure coding policy.\n\n" +
+          listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      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 DistributedFileSystem dfs = AdminHelper.getDFS(conf);
+      try {
+        dfs.disableErasureCodingPolicy(ecPolicyName);
+        System.out.println("Erasure coding policy " + ecPolicyName +
+            " is disabled");
+      } catch (IOException e) {
+        System.err.println(AdminHelper.prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
 
   private static final AdminHelper.Command[] COMMANDS = {
       new ListECPoliciesCommand(),
@@ -471,6 +566,8 @@ public class ECAdmin extends Configured implements Tool {
       new RemoveECPolicyCommand(),
       new SetECPolicyCommand(),
       new UnsetECPolicyCommand(),
-      new ListECCodecsCommand()
+      new ListECCodecsCommand(),
+      new EnableECPolicyCommand(),
+      new DisableECPolicyCommand()
   };
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 38bdc40..5903a36 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -446,6 +446,8 @@ Usage:
          [-listPolicies]
          [-addPolicies -policyFile <file>]
          [-listCodecs]
+         [-enablePolicy -policy <policyName>]
+         [-disablePolicy -policy <policyName>]
          [-help [cmd ...]]
 
 | COMMAND\_OPTION | Description |
@@ -456,6 +458,8 @@ Usage:
 |-listPolicies| Lists all supported ErasureCoding policies|
 |-addPolicies| Add a list of erasure coding policies|
 |-listCodecs| Get the list of supported erasure coding codecs and coders in system|
+|-enablePolicy| Enable an ErasureCoding policy in system|
+|-disablePolicy| Disable an ErasureCoding policy in system|
 
 Runs the ErasureCoding CLI. See [HDFS ErasureCoding](./HDFSErasureCoding.html#Administrative_commands) for more information on this command.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/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 4d0b2e6..69e8ef2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -156,6 +156,8 @@ Deployment
          [-listPolicies]
          [-addPolicies -policyFile <file>]
          [-listCodecs]
+         [-enablePolicy -policy <policyName>]
+         [-disablePolicy -policy <policyName>]
          [-help [cmd ...]]
 
 Below are the details about each command.
@@ -190,4 +192,12 @@ Below are the details about each command.
 
 *  `[-removePolicy -policy <policyName>]`
 
-     Remove an erasure coding policy.
\ No newline at end of file
+     Remove an erasure coding policy.
+
+*  `[-enablePolicy -policy <policyName>]`
+
+     Enable an erasure coding policy.
+
+*  `[-disablePolicy -policy <policyName>]`
+
+     Disable an erasure coding policy.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index 1d24f52..b35d374 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -1567,4 +1567,52 @@ public class TestDistributedFileSystem {
       }
     }
   }
+
+  @Test
+  public void testEnableAndDisableErasureCodingPolicy() throws Exception {
+    Configuration conf = getTestConfiguration();
+    MiniDFSCluster cluster = null;
+
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      ECSchema toAddSchema = new ECSchema("rs", 3, 2);
+      ErasureCodingPolicy toAddPolicy =
+          new ErasureCodingPolicy(toAddSchema, 128 * 1024, (byte) 254);
+      String policyName = toAddPolicy.getName();
+      ErasureCodingPolicy[] policies =
+          new ErasureCodingPolicy[]{toAddPolicy};
+      fs.addErasureCodingPolicies(policies);
+      assertEquals(policyName, ErasureCodingPolicyManager.getInstance().
+          getByName(policyName).getName());
+      fs.disableErasureCodingPolicy(policyName);
+      assertEquals(policyName, ErasureCodingPolicyManager.getInstance().
+          getRemovedPolicies().get(0).getName());
+      fs.enableErasureCodingPolicy(policyName);
+      assertEquals(policyName, ErasureCodingPolicyManager.getInstance().
+          getByName(policyName).getName());
+
+      //test enable a policy that doesn't exist
+      try {
+        fs.enableErasureCodingPolicy("notExistECName");
+        Assert.fail("enable the policy that doesn't exist should fail");
+      } catch (Exception e) {
+        GenericTestUtils.assertExceptionContains("does not exists", e);
+        // pass
+      }
+
+      //test disable a policy that doesn't exist
+      try {
+        fs.disableErasureCodingPolicy("notExistECName");
+        Assert.fail("disable the policy that doesn't exist should fail");
+      } catch (Exception e) {
+        GenericTestUtils.assertExceptionContains("does not exists", e);
+        // pass
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f99b6d19/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 26acc1f..791e685 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -163,6 +163,44 @@
       </comparators>
     </test>
 
+    <test>
+      <description>help: enablePolicy command</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -help enablePolicy</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Enable the erasure coding policy</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>[-enablePolicy -policy &lt;policy&gt;]</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>help: disablePolicy command</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -help disablePolicy</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Disable the erasure coding policy</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>[-disablePolicy -policy &lt;policy&gt;]</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
   <!-- Test erasure code commands -->
     <test>
       <description>setPolicy : set erasure coding policy on a directory to encode files</description>
@@ -349,6 +387,70 @@
       </comparators>
     </test>
 
+    <test>
+      <description>enablePolicy : enable the erasure coding policy</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -enablePolicy -policy RS-6-3-64k</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Erasure coding policy RS-6-3-64k is enabled</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>enablePolicy : enable the erasure coding policy twice</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -enablePolicy -policy RS-6-3-64k</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -enablePolicy -policy RS-6-3-64k</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Erasure coding policy RS-6-3-64k is enabled</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>disablePolicy : disable the erasure coding policy</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -disablePolicy -policy RS-6-3-64k</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <ec-admin-command>-fs NAMENODE -enablePolicy -policy RS-6-3-64k</ec-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Erasure coding policy RS-6-3-64k is disabled</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>disablePolicy : disable the erasure coding policy twice</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -disablePolicy -policy RS-6-3-64k</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -disablePolicy -policy RS-6-3-64k</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <ec-admin-command>-fs NAMENODE -enablePolicy -policy RS-6-3-64k</ec-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Erasure coding policy RS-6-3-64k is disabled</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
 <!-- Test illegal parameters -->
     <test>
       <description>setPolicy : illegal parameters - path is missing</description>
@@ -542,6 +644,66 @@
     </test>
 
     <test>
+      <description>enablePolicy : illegal parameters - policy is missing</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -enablePolicy RS-6-3-64k</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Please specify the policy name</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>enablePolicy : illegal parameters - too many parameters</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -enablePolicy -policy RS-6-3-64k RS-3-2-64k</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-enablePolicy: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>disablePolicy : illegal parameters - policy is missing</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -disablePolicy RS-6-3-64k</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Please specify the policy name</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>disablePolicy : illegal parameters - too many parameters</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -disablePolicy -policy RS-6-3-64k RS-3-2-64k</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-disablePolicy: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>listCodecs : illegal parameters - too many parameters</description>
       <test-commands>
         <ec-admin-command>-fs NAMENODE -listCodecs /ecdir</ec-admin-command>


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