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 ay...@apache.org on 2020/09/06 02:38:35 UTC

[hadoop] 01/01: HDFS-15117. EC: Add getECTopologyResultForPolicies to DistributedFileSystem. Contributed by Ayush Saxena. (#2261)

This is an automated email from the ASF dual-hosted git repository.

ayushsaxena pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 544d6da3ab01789a7224e0f796a6caf5e0f51a4a
Author: Ayush Saxena <ay...@apache.org>
AuthorDate: Sun Aug 30 10:19:26 2020 +0530

    HDFS-15117. EC: Add getECTopologyResultForPolicies to DistributedFileSystem. Contributed by Ayush Saxena. (#2261)
---
 .../java/org/apache/hadoop/hdfs/DFSClient.java     |  12 ++
 .../apache/hadoop/hdfs/DistributedFileSystem.java  |  13 +++
 .../hadoop/hdfs/protocol/ClientProtocol.java       |  12 ++
 .../hdfs/protocol}/ECTopologyVerifierResult.java   |   2 +-
 .../ClientNamenodeProtocolTranslatorPB.java        |  25 +++-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java     |  16 +++
 .../src/main/proto/ClientNamenodeProtocol.proto    |   2 +
 .../src/main/proto/erasurecoding.proto             |   8 ++
 .../hadoop-hdfs-client/src/main/proto/hdfs.proto   |   5 +
 .../apache/hadoop/hdfs/protocol/TestReadOnly.java  |   3 +-
 .../server/federation/router/ErasureCoding.java    |  22 ++++
 .../federation/router/RouterClientProtocol.java    |   8 ++
 .../server/federation/router/RouterRpcServer.java  |   7 ++
 .../server/federation/MiniRouterDFSCluster.java    |  12 ++
 .../federation/router/TestRouterMultiRack.java     | 129 +++++++++++++++++++++
 ...ientNamenodeProtocolServerSideTranslatorPB.java |  21 ++++
 .../hdfs/server/common/ECTopologyVerifier.java     |  19 ++-
 .../hdfs/server/namenode/FSDirErasureCodingOp.java |  27 ++++-
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  |  64 ++++++++--
 .../hdfs/server/namenode/NameNodeRpcServer.java    |   7 ++
 .../java/org/apache/hadoop/hdfs/tools/ECAdmin.java |  87 +++-----------
 .../hadoop/hdfs/TestDistributedFileSystem.java     |  32 +++++
 22 files changed, 438 insertions(+), 95 deletions(-)

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 187143e..c3a00c9 100755
--- 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
@@ -116,6 +116,7 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@@ -2770,6 +2771,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  public ECTopologyVerifierResult getECTopologyResultForPolicies(
+      final String... policyNames) throws IOException {
+    checkOpen();
+    try {
+      return namenode.getECTopologyResultForPolicies(policyNames);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class);
+    }
+  }
+
   public void setXAttr(String src, String name, byte[] value,
       EnumSet<XAttrSetFlag> flag) throws IOException {
     checkOpen();
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 7a218bf..b040040 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
@@ -80,6 +80,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
@@ -3097,6 +3098,18 @@ public class DistributedFileSystem extends FileSystem
   }
 
   /**
+   * Verifies if the given policies are supported in the given cluster setup.
+   * If not policy is specified checks for all enabled policies.
+   * @param policyNames name of policies.
+   * @return the result if the given policies are supported in the cluster setup
+   * @throws IOException
+   */
+  public ECTopologyVerifierResult getECTopologyResultForPolicies(
+      final String... policyNames) throws IOException {
+    return dfs.getECTopologyResultForPolicies(policyNames);
+  }
+
+  /**
    * Get the root directory of Trash for a path in HDFS.
    * 1. File in encryption zone returns /ez1/.Trash/username
    * 2. File not in encryption zone, or encountered exception when checking
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 953e48a..dae77a4 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
@@ -1742,6 +1742,18 @@ public interface ClientProtocol {
   void unsetErasureCodingPolicy(String src) throws IOException;
 
   /**
+   * Verifies if the given policies are supported in the given cluster setup.
+   * If not policy is specified checks for all enabled policies.
+   * @param policyNames name of policies.
+   * @return the result if the given policies are supported in the cluster setup
+   * @throws IOException
+   */
+  @Idempotent
+  @ReadOnly
+  ECTopologyVerifierResult getECTopologyResultForPolicies(String... policyNames)
+      throws IOException;
+
+  /**
    * Get {@link QuotaUsage} rooted at the specified directory.
    *
    * Note: due to HDFS-6763, standby/observer doesn't keep up-to-date info
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECTopologyVerifierResult.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECTopologyVerifierResult.java
similarity index 96%
rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECTopologyVerifierResult.java
rename to hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECTopologyVerifierResult.java
index 4757eab..159688c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECTopologyVerifierResult.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ECTopologyVerifierResult.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.protocol;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 
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 65ebc2c..af4abb0 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
@@ -64,6 +64,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
@@ -216,6 +217,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodin
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCodingPolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CodecProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECTopologyResultForPoliciesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECTopologyResultForPoliciesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
@@ -1611,10 +1614,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void unsetErasureCodingPolicy(String src)
-      throws IOException {
+  public void unsetErasureCodingPolicy(String src) throws IOException {
     final UnsetErasureCodingPolicyRequestProto.Builder builder =
-        ErasureCodingProtos.UnsetErasureCodingPolicyRequestProto.newBuilder();
+        UnsetErasureCodingPolicyRequestProto.newBuilder();
     builder.setSrc(src);
     UnsetErasureCodingPolicyRequestProto req = builder.build();
     try {
@@ -1625,6 +1627,23 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
+  public ECTopologyVerifierResult getECTopologyResultForPolicies(
+      final String... policyNames) throws IOException {
+    final GetECTopologyResultForPoliciesRequestProto.Builder builder =
+        GetECTopologyResultForPoliciesRequestProto.newBuilder();
+    builder.addAllPolicies(Arrays.asList(policyNames));
+    GetECTopologyResultForPoliciesRequestProto req = builder.build();
+    try {
+      GetECTopologyResultForPoliciesResponseProto response =
+          rpcProxy.getECTopologyResultForPolicies(null, req);
+      return PBHelperClient
+          .convertECTopologyVerifierResultProto(response.getResponse());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
   public void reencryptEncryptionZone(String zone, ReencryptAction action)
       throws IOException {
     final ReencryptEncryptionZoneRequestProto.Builder builder =
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 3d43c97..cf1a92f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -84,6 +84,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
@@ -3337,6 +3338,21 @@ public class PBHelperClient {
     return builder.build();
   }
 
+  public static ECTopologyVerifierResult convertECTopologyVerifierResultProto(
+      HdfsProtos.ECTopologyVerifierResultProto resp) {
+    return new ECTopologyVerifierResult(resp.getIsSupported(),
+        resp.getResultMessage());
+  }
+
+  public static HdfsProtos.ECTopologyVerifierResultProto convertECTopologyVerifierResult(
+      ECTopologyVerifierResult resp) {
+    final HdfsProtos.ECTopologyVerifierResultProto.Builder builder =
+        HdfsProtos.ECTopologyVerifierResultProto.newBuilder()
+            .setIsSupported(resp.isSupported())
+            .setResultMessage(resp.getResultMessage());
+    return builder.build();
+  }
+
   public static EnumSet<AddBlockFlag> convertAddBlockFlags(
       List<AddBlockFlagProto> addBlockFlags) {
     EnumSet<AddBlockFlag> flags =
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 ce78d1f..f353c03 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
@@ -1016,6 +1016,8 @@ service ClientNamenodeProtocol {
       returns(SetErasureCodingPolicyResponseProto);
   rpc unsetErasureCodingPolicy(UnsetErasureCodingPolicyRequestProto)
       returns(UnsetErasureCodingPolicyResponseProto);
+  rpc getECTopologyResultForPolicies(GetECTopologyResultForPoliciesRequestProto)
+      returns(GetECTopologyResultForPoliciesResponseProto);
   rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto)
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
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 de3bf4a..d92dd4c 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
@@ -89,6 +89,14 @@ message UnsetErasureCodingPolicyRequestProto {
 message UnsetErasureCodingPolicyResponseProto {
 }
 
+message GetECTopologyResultForPoliciesRequestProto {
+  repeated string policies = 1;
+}
+
+message GetECTopologyResultForPoliciesResponseProto {
+  required ECTopologyVerifierResultProto response = 1;
+}
+
 /**
  * Block erasure coding reconstruction info
  */
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index 1f0e179..3845934 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -412,6 +412,11 @@ message AddErasureCodingPolicyResponseProto {
   optional string errorMsg = 3;
 }
 
+message ECTopologyVerifierResultProto {
+  required string resultMessage = 1;
+  required bool isSupported = 2;
+}
+
 /**
  * Placeholder type for consistent HDFS operations.
  */
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java
index e0432f5..3454db9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java
@@ -73,7 +73,8 @@ public class TestReadOnly {
           "getEditsFromTxid",
           "getQuotaUsage",
           "msync",
-          "getHAServiceState"
+          "getHAServiceState",
+          "getECTopologyResultForPolicies"
       )
   );
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
index 480b232..5e6fa27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
@@ -28,6 +28,7 @@ import java.util.Set;
 
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
@@ -170,6 +171,27 @@ public class ErasureCoding {
     rpcClient.invokeSequential(locations, remoteMethod, null, null);
   }
 
+  public ECTopologyVerifierResult getECTopologyResultForPolicies(
+      String[] policyNames) throws IOException {
+    RemoteMethod method = new RemoteMethod("getECTopologyResultForPolicies",
+        new Class<?>[] {String[].class}, new Object[] {policyNames});
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    if (nss.isEmpty()) {
+      throw new IOException("No namespace availaible.");
+    }
+    Map<FederationNamespaceInfo, ECTopologyVerifierResult> ret = rpcClient
+        .invokeConcurrent(nss, method, true, false,
+            ECTopologyVerifierResult.class);
+    for (Map.Entry<FederationNamespaceInfo, ECTopologyVerifierResult> entry : ret
+        .entrySet()) {
+      if (!entry.getValue().isSupported()) {
+        return entry.getValue();
+      }
+    }
+    // If no negative result, return the result from the first namespace.
+    return ret.get(nss.iterator().next());
+  }
+
   public ECBlockGroupStats getECBlockGroupStats() throws IOException {
     rpcServer.checkOperation(OperationCategory.READ);
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
index 57f0584..8455f4b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
@@ -1508,6 +1509,13 @@ public class RouterClientProtocol implements ClientProtocol {
   }
 
   @Override
+  public ECTopologyVerifierResult getECTopologyResultForPolicies(
+      String... policyNames) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED, true);
+    return erasureCoding.getECTopologyResultForPolicies(policyNames);
+  }
+
+  @Override
   public ECBlockGroupStats getECBlockGroupStats() throws IOException {
     return erasureCoding.getECBlockGroupStats();
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index c7b4b58..ad9b187 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
@@ -1169,6 +1170,12 @@ public class RouterRpcServer extends AbstractService
     clientProto.unsetErasureCodingPolicy(src);
   }
 
+  @Override
+  public ECTopologyVerifierResult getECTopologyResultForPolicies(
+      String... policyNames) throws IOException {
+    return clientProto.getECTopologyResultForPolicies(policyNames);
+  }
+
   @Override // ClientProtocol
   public ECBlockGroupStats getECBlockGroupStats() throws IOException {
     return clientProto.getECBlockGroupStats();
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java
index e34713d..7b59e3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java
@@ -570,6 +570,17 @@ public class MiniRouterDFSCluster {
     this.numDatanodesPerNameservice = num;
   }
 
+  /** Racks for datanodes. */
+  private String[] racks = null;
+
+  /**
+   * Set racks for each datanode. If racks is uninitialized or passed null then
+   * default is used.
+   */
+  public void setRacks(String[] racks) {
+    this.racks = racks;
+  }
+
   /**
    * Set the DNs to belong to only one subcluster.
    */
@@ -723,6 +734,7 @@ public class MiniRouterDFSCluster {
           .numDataNodes(numDNs)
           .nnTopology(topology)
           .dataNodeConfOverlays(dnConfs)
+          .racks(racks)
           .build();
       cluster.waitActive();
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMultiRack.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMultiRack.java
new file mode 100644
index 0000000..540a123
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMultiRack.java
@@ -0,0 +1,129 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.server.federation.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.*;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
+import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.NamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.resolver.MultipleDestinationMountTableResolver;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class with clusters having multiple racks.
+ */
+public class TestRouterMultiRack {
+  private static StateStoreDFSCluster cluster;
+  private static RouterContext routerContext;
+  private static DistributedFileSystem routerFs;
+  private static NamenodeContext nnContext0;
+  private static NamenodeContext nnContext1;
+  private static DistributedFileSystem nnFs0;
+  private static DistributedFileSystem nnFs1;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+
+    // Build and start a federated cluster
+    cluster = new StateStoreDFSCluster(false, 2,
+        MultipleDestinationMountTableResolver.class);
+    Configuration routerConf =
+        new RouterConfigBuilder().stateStore().admin().quota().rpc().build();
+    Configuration hdfsConf = new Configuration(false);
+    cluster.addNamenodeOverrides(hdfsConf);
+    cluster.addRouterOverrides(routerConf);
+    cluster.setNumDatanodesPerNameservice(9);
+    cluster.setIndependentDNs();
+    cluster.setRacks(
+        new String[] {"/rack1", "/rack1", "/rack1", "/rack2", "/rack2",
+            "/rack2", "/rack3", "/rack3", "/rack3", "/rack4", "/rack4",
+            "/rack4", "/rack5", "/rack5", "/rack5", "/rack6", "/rack6",
+            "/rack6"});
+    cluster.startCluster();
+    cluster.startRouters();
+    cluster.waitClusterUp();
+
+    routerContext = cluster.getRandomRouter();
+    routerFs = (DistributedFileSystem) routerContext.getFileSystem();
+    nnContext0 = cluster.getNamenode("ns0", null);
+    nnContext1 = cluster.getNamenode("ns1", null);
+    nnFs0 = (DistributedFileSystem) nnContext0.getFileSystem();
+    nnFs1 = (DistributedFileSystem) nnContext1.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    if (cluster != null) {
+      cluster.stopRouter(routerContext);
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  @Test
+  public void testGetECTopologyResultForPolicies() throws IOException {
+    routerFs.enableErasureCodingPolicy("RS-6-3-1024k");
+    // No policies specified should return result for the enabled policy.
+    ECTopologyVerifierResult result = routerFs.getECTopologyResultForPolicies();
+    assertTrue(result.isSupported());
+    // Specified policy requiring more datanodes than present in
+    // the actual cluster.
+    result = routerFs.getECTopologyResultForPolicies("RS-10-4-1024k");
+    assertFalse(result.isSupported());
+    // Specify multiple policies with one policy requiring more datanodes than
+    // present in the actual cluster
+    result = routerFs
+        .getECTopologyResultForPolicies("RS-10-4-1024k", "RS-3-2-1024k");
+    assertFalse(result.isSupported());
+    // Specify multiple policies that require datanodes equal or less then
+    // present in the actual cluster
+    result = routerFs
+        .getECTopologyResultForPolicies("XOR-2-1-1024k", "RS-3-2-1024k");
+    assertTrue(result.isSupported());
+    // Specify multiple policies with one policy requiring more datanodes than
+    // present in the actual cluster
+    result = routerFs
+        .getECTopologyResultForPolicies("RS-10-4-1024k", "RS-3-2-1024k");
+    assertFalse(result.isSupported());
+    // Enable a policy requiring more datanodes than present in
+    // the actual cluster.
+    routerFs.enableErasureCodingPolicy("RS-10-4-1024k");
+    result = routerFs.getECTopologyResultForPolicies();
+    assertFalse(result.isSupported());
+    // Check without specifying any policy, with one cluster having
+    // all supported, but one cluster having one unsupported policy. The
+    nnFs0.disableErasureCodingPolicy("RS-10-4-1024k");
+    nnFs1.enableErasureCodingPolicy("RS-10-4-1024k");
+    result = routerFs.getECTopologyResultForPolicies();
+    assertFalse(result.isSupported());
+    nnFs1.disableErasureCodingPolicy("RS-10-4-1024k");
+    nnFs0.enableErasureCodingPolicy("RS-10-4-1024k");
+    result = routerFs.getECTopologyResultForPolicies();
+    assertFalse(result.isSupported());
+  }
+}
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 abeec1c..c6871d6 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
@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
@@ -262,6 +263,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCod
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCodingPolicyResponseProto;
 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.GetECTopologyResultForPoliciesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECTopologyResultForPoliciesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.*;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
@@ -1622,6 +1625,24 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+  public GetECTopologyResultForPoliciesResponseProto getECTopologyResultForPolicies(
+      RpcController controller, GetECTopologyResultForPoliciesRequestProto req)
+      throws ServiceException {
+    try {
+      List<String> policies = req.getPoliciesList();
+      ECTopologyVerifierResult result = server.getECTopologyResultForPolicies(
+          policies.toArray(policies.toArray(new String[policies.size()])));
+      GetECTopologyResultForPoliciesResponseProto.Builder builder =
+          GetECTopologyResultForPoliciesResponseProto.newBuilder();
+      builder
+          .setResponse(PBHelperClient.convertECTopologyVerifierResult(result));
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public SetXAttrResponseProto setXAttr(RpcController controller,
       SetXAttrRequestProto req) throws ServiceException {
     try {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/ECTopologyVerifier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/ECTopologyVerifier.java
index 66c7c4e..621ebff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/ECTopologyVerifier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/ECTopologyVerifier.java
@@ -18,12 +18,12 @@ package org.apache.hadoop.hdfs.server.common;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.server.namenode.ECTopologyVerifierResult;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.stream.Collectors;
@@ -52,7 +52,8 @@ public final class ECTopologyVerifier {
    * @return the status of the verification
    */
   public static ECTopologyVerifierResult getECTopologyVerifierResult(
-      final DatanodeInfo[] report, final ErasureCodingPolicy... policies) {
+      final DatanodeInfo[] report,
+      final Collection<ErasureCodingPolicy> policies) {
     final int numOfRacks = getNumberOfRacks(report);
     return getECTopologyVerifierResult(numOfRacks, report.length, policies);
   }
@@ -60,14 +61,14 @@ public final class ECTopologyVerifier {
   /**
    * Verifies whether the cluster setup can support all enabled EC policies.
    *
-   * @param policies erasure coding policies to verify
    * @param numOfRacks number of racks
    * @param numOfDataNodes number of data nodes
+   * @param policies erasure coding policies to verify
    * @return the status of the verification
    */
   public static ECTopologyVerifierResult getECTopologyVerifierResult(
       final int numOfRacks, final int numOfDataNodes,
-      final ErasureCodingPolicy... policies) {
+      final Collection<ErasureCodingPolicy> policies) {
     int minDN = 0;
     int minRack = 0;
     for (ErasureCodingPolicy policy: policies) {
@@ -126,10 +127,8 @@ public final class ECTopologyVerifier {
   }
 
   private static String getReadablePolicies(
-      final ErasureCodingPolicy... policies) {
-    return Arrays.asList(policies)
-            .stream()
-            .map(policyInfo -> policyInfo.getName())
-            .collect(Collectors.joining(", "));
+      final Collection<ErasureCodingPolicy> policies) {
+    return policies.stream().map(policyInfo -> policyInfo.getName())
+        .collect(Collectors.joining(", "));
   }
 }
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 b0bc5e4..011c72e 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
@@ -68,7 +68,7 @@ final class FSDirErasureCodingOp {
    * @return an erasure coding policy if ecPolicyName is valid and enabled
    * @throws IOException
    */
-  static ErasureCodingPolicy getErasureCodingPolicyByName(
+  static ErasureCodingPolicy getEnabledErasureCodingPolicyByName(
       final FSNamesystem fsn, final String ecPolicyName) throws IOException {
     assert fsn.hasReadLock();
     ErasureCodingPolicy ecPolicy = fsn.getErasureCodingPolicyManager()
@@ -93,6 +93,27 @@ final class FSDirErasureCodingOp {
   }
 
   /**
+   * Check if the ecPolicyName is valid, return the corresponding
+   * EC policy if is, including the REPLICATION EC policy.
+   * @param fsn namespace
+   * @param ecPolicyName name of EC policy to be checked
+   * @return an erasure coding policy if ecPolicyName is valid
+   * @throws IOException
+   */
+  static ErasureCodingPolicy getErasureCodingPolicyByName(
+      final FSNamesystem fsn, final String ecPolicyName) throws IOException {
+    assert fsn.hasReadLock();
+    ErasureCodingPolicy ecPolicy = fsn.getErasureCodingPolicyManager()
+        .getErasureCodingPolicyByName(ecPolicyName);
+    if (ecPolicy == null) {
+      throw new HadoopIllegalArgumentException(
+          "The given erasure coding " + "policy " + ecPolicyName
+              + " does not exist.");
+    }
+    return ecPolicy;
+  }
+
+  /**
    * Set an erasure coding policy on the given path.
    *
    * @param fsn The namespace
@@ -118,7 +139,7 @@ final class FSDirErasureCodingOp {
     List<XAttr> xAttrs;
     fsd.writeLock();
     try {
-      ErasureCodingPolicy ecPolicy = getErasureCodingPolicyByName(fsn,
+      ErasureCodingPolicy ecPolicy = getEnabledErasureCodingPolicyByName(fsn,
           ecPolicyName);
       iip = fsd.resolvePath(pc, src, DirOp.WRITE_LINK);
       // Write access is required to set erasure coding policy
@@ -374,7 +395,7 @@ final class FSDirErasureCodingOp {
       String ecPolicyName, INodesInPath iip) throws IOException {
     ErasureCodingPolicy ecPolicy;
     if (!StringUtils.isEmpty(ecPolicyName)) {
-      ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicyByName(
+      ecPolicy = FSDirErasureCodingOp.getEnabledErasureCodingPolicyByName(
           fsn, ecPolicyName);
     } else {
       ecPolicy = FSDirErasureCodingOp.unprotectedGetErasureCodingPolicy(
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 ff88663..a937998 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
@@ -91,6 +91,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LI
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT;
 
 import java.util.concurrent.atomic.AtomicLong;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
@@ -7768,6 +7769,48 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
+   * Verifies if the given policies are supported in the given cluster setup.
+   * If not policy is specified checks for all enabled policies.
+   * @param policyNames name of policies.
+   * @return the result if the given policies are supported in the cluster setup
+   * @throws IOException
+   */
+  public ECTopologyVerifierResult getECTopologyResultForPolicies(
+      String[] policyNames) throws IOException {
+    String operationName = "getECTopologyResultForPolicies";
+    checkSuperuserPrivilege(operationName);
+    checkOperation(OperationCategory.UNCHECKED);
+    ECTopologyVerifierResult result;
+    readLock();
+    try {
+      checkOperation(OperationCategory.UNCHECKED);
+      // If no policy name is specified return the result
+      // for all enabled policies.
+      if (policyNames == null || policyNames.length == 0) {
+        result = getEcTopologyVerifierResultForEnabledPolicies();
+      } else {
+        Collection<ErasureCodingPolicy> policies =
+            new ArrayList<ErasureCodingPolicy>();
+        for (int i = 0; i < policyNames.length; i++) {
+          policies.add(FSDirErasureCodingOp
+              .getErasureCodingPolicyByName(this, policyNames[i]));
+        }
+        int numOfDataNodes =
+            getBlockManager().getDatanodeManager().getNumOfDataNodes();
+        int numOfRacks =
+            getBlockManager().getDatanodeManager().getNetworkTopology()
+                .getNumOfRacks();
+        result = ECTopologyVerifier
+            .getECTopologyVerifierResult(numOfRacks, numOfDataNodes, policies);
+      }
+    } finally {
+      readUnlock();
+    }
+    logAuditEvent(true, operationName, null);
+    return result;
+  }
+
+  /**
    * Get the erasure coding policy information for specified path
    */
   ErasureCodingPolicy getErasureCodingPolicy(String src)
@@ -8194,15 +8237,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   @Override // NameNodeMXBean
   public String getVerifyECWithTopologyResult() {
-    int numOfDataNodes = getBlockManager().getDatanodeManager()
-        .getNumOfDataNodes();
-    int numOfRacks = getBlockManager().getDatanodeManager()
-        .getNetworkTopology().getNumOfRacks();
-    ErasureCodingPolicy[] enabledEcPolicies =
-        getErasureCodingPolicyManager().getCopyOfEnabledPolicies();
     ECTopologyVerifierResult result =
-        ECTopologyVerifier.getECTopologyVerifierResult(
-            numOfRacks, numOfDataNodes, enabledEcPolicies);
+        getEcTopologyVerifierResultForEnabledPolicies();
 
     Map<String, String> resultMap = new HashMap<String, String>();
     resultMap.put("isSupported", Boolean.toString(result.isSupported()));
@@ -8210,6 +8246,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return JSON.toString(resultMap);
   }
 
+  private ECTopologyVerifierResult getEcTopologyVerifierResultForEnabledPolicies() {
+    int numOfDataNodes =
+        getBlockManager().getDatanodeManager().getNumOfDataNodes();
+    int numOfRacks = getBlockManager().getDatanodeManager().getNetworkTopology()
+        .getNumOfRacks();
+    ErasureCodingPolicy[] enabledEcPolicies =
+        getErasureCodingPolicyManager().getCopyOfEnabledPolicies();
+    return ECTopologyVerifier
+        .getECTopologyVerifierResult(numOfRacks, numOfDataNodes,
+            Arrays.asList(enabledEcPolicies));
+  }
+
   // This method logs operatoinName without super user privilege.
   // It should be called without holding FSN lock.
   void checkSuperuserPrivilege(String operationName)
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 ff1fea1..3a8cc40 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
@@ -101,6 +101,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
@@ -2485,6 +2486,12 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override
+  public ECTopologyVerifierResult getECTopologyResultForPolicies(
+      String... policyNames) throws IOException {
+    return namesystem.getECTopologyResultForPolicies(policyNames);
+  }
+
+  @Override
   public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
     checkNNStartup();
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 80d1b6f..e499799 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
@@ -26,15 +26,13 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.NoECPolicySetException;
-import org.apache.hadoop.hdfs.server.common.ECTopologyVerifier;
-import org.apache.hadoop.hdfs.server.namenode.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.util.ECPolicyLoader;
 import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
@@ -536,14 +534,14 @@ public class ECAdmin extends Configured implements Tool {
       final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
         dfs.enableErasureCodingPolicy(ecPolicyName);
-        System.out.println("Erasure coding policy " + ecPolicyName +
-            " is enabled");
+        System.out
+            .println("Erasure coding policy " + ecPolicyName + " is enabled");
         ECTopologyVerifierResult result =
-            getECTopologyVerifierResultForPolicy(dfs, ecPolicyName);
+            dfs.getECTopologyResultForPolicies(ecPolicyName);
         if (!result.isSupported()) {
-          System.err.println("Warning: The cluster setup does not support " +
-              "EC policy " + ecPolicyName + ". Reason: " +
-              result.getResultMessage());
+          System.err.println(
+              "Warning: The cluster setup does not support " + "EC policy "
+                  + ecPolicyName + ". Reason: " + result.getResultMessage());
         }
       } catch (IOException e) {
         System.err.println(AdminHelper.prettifyException(e));
@@ -630,14 +628,21 @@ public class ECAdmin extends Configured implements Tool {
     public int run(Configuration conf, List<String> args) throws IOException {
       boolean isPolicyOption = StringUtils.popOption("-policy", args);
       final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
-      ECTopologyVerifierResult result;
+      ECTopologyVerifierResult result = null;
       if (isPolicyOption) {
         CommandFormat c = new CommandFormat(1, Integer.MAX_VALUE);
         c.parse(args);
         String[] parameters = args.toArray(new String[args.size()]);
-        result = getECTopologyResultForPolicies(dfs, parameters);
+        try {
+          result = dfs.getECTopologyResultForPolicies(parameters);
+        } catch (RemoteException e) {
+          if (e.getClassName().contains("HadoopIllegalArgumentException")) {
+            throw new HadoopIllegalArgumentException(e.getMessage());
+          }
+          throw e;
+        }
       } else {
-        result = getECTopologyVerifierResult(dfs);
+        result = dfs.getECTopologyResultForPolicies();
       }
       System.out.println(result.getResultMessage());
       if (result.isSupported()) {
@@ -647,62 +652,6 @@ public class ECAdmin extends Configured implements Tool {
     }
   }
 
-  private static ECTopologyVerifierResult getECTopologyVerifierResult(
-      final DistributedFileSystem dfs) throws IOException {
-    final ErasureCodingPolicyInfo[] policies =
-        dfs.getClient().getNamenode().getErasureCodingPolicies();
-    final DatanodeInfo[] report = dfs.getClient().getNamenode()
-        .getDatanodeReport(HdfsConstants.DatanodeReportType.ALL);
-
-    return ECTopologyVerifier.getECTopologyVerifierResult(report,
-        getEnabledPolicies(policies));
-  }
-
-  private static ECTopologyVerifierResult getECTopologyResultForPolicies(
-      final DistributedFileSystem dfs, final String... policyNames)
-      throws IOException {
-    ErasureCodingPolicy[] policies =
-        new ErasureCodingPolicy[policyNames.length];
-    for (int i = 0; i < policyNames.length; i++) {
-      policies[i] =
-        getPolicy(dfs.getClient().getNamenode().getErasureCodingPolicies(),
-            policyNames[i]);
-    }
-    final DatanodeInfo[] report = dfs.getClient().getNamenode()
-        .getDatanodeReport(HdfsConstants.DatanodeReportType.ALL);
-    return ECTopologyVerifier.getECTopologyVerifierResult(report, policies);
-  }
-
-  private static ECTopologyVerifierResult getECTopologyVerifierResultForPolicy(
-      final DistributedFileSystem dfs, final String policyName)
-      throws IOException {
-    final ErasureCodingPolicy policy =
-        getPolicy(dfs.getClient().getNamenode().getErasureCodingPolicies(),
-            policyName);
-    final DatanodeInfo[] report = dfs.getClient().getNamenode()
-        .getDatanodeReport(HdfsConstants.DatanodeReportType.ALL);
-    return ECTopologyVerifier.getECTopologyVerifierResult(report, policy);
-  }
-
-  private static ErasureCodingPolicy getPolicy(
-      final ErasureCodingPolicyInfo[] policies, final String policyName) {
-    for (ErasureCodingPolicyInfo policy : policies) {
-      if (policyName.equals(policy.getPolicy().getName())) {
-        return policy.getPolicy();
-      }
-    }
-    throw new HadoopIllegalArgumentException("The given erasure coding " +
-        "policy " + policyName + " does not exist.");
-  }
-
-  private static ErasureCodingPolicy[] getEnabledPolicies(
-      final ErasureCodingPolicyInfo[] policies) {
-    return Arrays.asList(policies).stream()
-        .filter(policyInfo -> policyInfo.isEnabled())
-        .map(ErasureCodingPolicyInfo::getPolicy)
-        .toArray(ErasureCodingPolicy[]::new);
-  }
-
   private static final AdminHelper.Command[] COMMANDS = {
       new ListECPoliciesCommand(),
       new AddECPoliciesCommand(),
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 8bef655..598cbf8 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
@@ -89,6 +89,7 @@ import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -2016,4 +2017,35 @@ public class TestDistributedFileSystem {
       assertEquals("Number of SSD should be 1 but was : " + numSSD, 1, numSSD);
     }
   }
+
+  @Test
+  public void testGetECTopologyResultForPolicies() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    try (MiniDFSCluster cluster = DFSTestUtil.setupCluster(conf, 9, 3, 0)) {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      dfs.enableErasureCodingPolicy("RS-6-3-1024k");
+      // No policies specified should return result for the enabled policy.
+      ECTopologyVerifierResult result = dfs.getECTopologyResultForPolicies();
+      assertTrue(result.isSupported());
+      // Specified policy requiring more datanodes than present in
+      // the actual cluster.
+      result = dfs.getECTopologyResultForPolicies("RS-10-4-1024k");
+      assertFalse(result.isSupported());
+      // Specify multiple policies that require datanodes equlal or less then
+      // present in the actual cluster
+      result =
+          dfs.getECTopologyResultForPolicies("XOR-2-1-1024k", "RS-3-2-1024k");
+      assertTrue(result.isSupported());
+      // Specify multiple policies with one policy requiring more datanodes than
+      // present in the actual cluster
+      result =
+          dfs.getECTopologyResultForPolicies("RS-10-4-1024k", "RS-3-2-1024k");
+      assertFalse(result.isSupported());
+      // Enable a policy requiring more datanodes than present in
+      // the actual cluster.
+      dfs.enableErasureCodingPolicy("RS-10-4-1024k");
+      result = dfs.getECTopologyResultForPolicies();
+      assertFalse(result.isSupported());
+    }
+  }
 }


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