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 cu...@apache.org on 2018/01/18 23:47:52 UTC

[21/49] hadoop git commit: HDFS-12919. RBF: Support erasure coding methods in RouterRpcServer. Contributed by Inigo Goiri.

HDFS-12919. RBF: Support erasure coding methods in RouterRpcServer. Contributed by Inigo Goiri.


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

Branch: refs/heads/YARN-7402
Commit: d5d6a0353bb85b882cc4ef60e3a12d63243d34ba
Parents: edcc3a9
Author: Inigo Goiri <in...@apache.org>
Authored: Fri Jan 12 16:25:39 2018 -0800
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri Jan 12 16:25:39 2018 -0800

----------------------------------------------------------------------
 .../AddErasureCodingPolicyResponse.java         |  24 ++
 .../server/federation/router/ErasureCoding.java | 198 +++++++++++++++
 .../hdfs/server/federation/router/Quota.java    |  10 +-
 .../federation/router/RouterRpcClient.java      |  65 ++++-
 .../federation/router/RouterRpcServer.java      | 241 ++++++++++---------
 .../server/federation/RouterDFSCluster.java     |  12 +-
 .../server/federation/router/TestRouterRpc.java | 191 ++++++++++++++-
 7 files changed, 604 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5d6a035/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
index 2e8d081..dc77a47 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 
 /**
@@ -65,4 +67,26 @@ public class AddErasureCodingPolicyResponse {
           + "error message is " + getErrorMsg();
     }
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof AddErasureCodingPolicyResponse) {
+      AddErasureCodingPolicyResponse other = (AddErasureCodingPolicyResponse) o;
+      return new EqualsBuilder()
+          .append(policy, other.policy)
+          .append(succeed, other.succeed)
+          .append(errorMsg, other.errorMsg)
+          .isEquals();
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(303855623, 582626729)
+        .append(policy)
+        .append(succeed)
+        .append(errorMsg)
+        .toHashCode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5d6a035/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
new file mode 100644
index 0000000..d2b2d50
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import static org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer.merge;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+
+/**
+ * Module that implements all the RPC calls in
+ * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol} related to
+ * Erasure Coding in the {@link RouterRpcServer}.
+ */
+public class ErasureCoding {
+
+  /** RPC server to receive client calls. */
+  private final RouterRpcServer rpcServer;
+  /** RPC clients to connect to the Namenodes. */
+  private final RouterRpcClient rpcClient;
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private final ActiveNamenodeResolver namenodeResolver;
+
+
+  public ErasureCoding(RouterRpcServer server) {
+    this.rpcServer = server;
+    this.rpcClient =  this.rpcServer.getRPCClient();
+    this.namenodeResolver = this.rpcClient.getNamenodeResolver();
+  }
+
+  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getErasureCodingPolicies");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, ErasureCodingPolicyInfo[]> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, ErasureCodingPolicyInfo[].class);
+    return merge(ret, ErasureCodingPolicyInfo.class);
+  }
+
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getErasureCodingCodecs");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    @SuppressWarnings("rawtypes")
+    Map<FederationNamespaceInfo, Map> retCodecs =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, Map.class);
+
+    Map<String, String> ret = new HashMap<>();
+    Object obj = retCodecs;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Map<String, String>> results =
+        (Map<FederationNamespaceInfo, Map<String, String>>)obj;
+    Collection<Map<String, String>> allCodecs = results.values();
+    for (Map<String, String> codecs : allCodecs) {
+      ret.putAll(codecs);
+    }
+
+    return ret;
+  }
+
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
+      ErasureCodingPolicy[] policies) throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("addErasureCodingPolicies",
+        new Class<?>[] {ErasureCodingPolicy[].class}, new Object[] {policies});
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, AddErasureCodingPolicyResponse[]> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, AddErasureCodingPolicyResponse[].class);
+
+    return merge(ret, AddErasureCodingPolicyResponse.class);
+  }
+
+  public void removeErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("removeErasureCodingPolicy",
+        new Class<?>[] {String.class}, ecPolicyName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("disableErasureCodingPolicy",
+        new Class<?>[] {String.class}, ecPolicyName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("enableErasureCodingPolicy",
+        new Class<?>[] {String.class}, ecPolicyName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod remoteMethod = new RemoteMethod("getErasureCodingPolicy",
+        new Class<?>[] {String.class}, new RemoteParam());
+    ErasureCodingPolicy ret = rpcClient.invokeSequential(
+        locations, remoteMethod, null, null);
+    return ret;
+  }
+
+  public void setErasureCodingPolicy(String src, String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod remoteMethod = new RemoteMethod("setErasureCodingPolicy",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), ecPolicyName);
+    rpcClient.invokeSequential(locations, remoteMethod, null, null);
+  }
+
+  public void unsetErasureCodingPolicy(String src) throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod remoteMethod = new RemoteMethod("unsetErasureCodingPolicy",
+        new Class<?>[] {String.class}, new RemoteParam());
+    rpcClient.invokeSequential(locations, remoteMethod, null, null);
+  }
+
+  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getECBlockGroupStats");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, ECBlockGroupStats> allStats =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, ECBlockGroupStats.class);
+
+    // Merge the stats from all the namespaces
+    long lowRedundancyBlockGroups = 0;
+    long corruptBlockGroups = 0;
+    long missingBlockGroups = 0;
+    long bytesInFutureBlockGroups = 0;
+    long pendingDeletionBlocks = 0;
+    for (ECBlockGroupStats stats : allStats.values()) {
+      lowRedundancyBlockGroups += stats.getLowRedundancyBlockGroups();
+      corruptBlockGroups += stats.getCorruptBlockGroups();
+      missingBlockGroups += stats.getMissingBlockGroups();
+      bytesInFutureBlockGroups += stats.getBytesInFutureBlockGroups();
+      pendingDeletionBlocks += stats.getPendingDeletionBlocks();
+    }
+    return new ECBlockGroupStats(lowRedundancyBlockGroups, corruptBlockGroups,
+        missingBlockGroups, bytesInFutureBlockGroups, pendingDeletionBlocks);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5d6a035/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java
index f5e5272..dbb6ffa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java
@@ -94,8 +94,8 @@ public class Quota {
     final List<RemoteLocation> quotaLocs = getValidQuotaLocations(path);
     RemoteMethod method = new RemoteMethod("getQuotaUsage",
         new Class<?>[] {String.class}, new RemoteParam());
-    Map<RemoteLocation, Object> results = rpcClient.invokeConcurrent(quotaLocs,
-        method, true, false);
+    Map<RemoteLocation, QuotaUsage> results = rpcClient.invokeConcurrent(
+        quotaLocs, method, true, false, QuotaUsage.class);
 
     return aggregateQuota(results);
   }
@@ -151,14 +151,14 @@ public class Quota {
    * @param results Quota query result.
    * @return Aggregated Quota.
    */
-  private QuotaUsage aggregateQuota(Map<RemoteLocation, Object> results) {
+  private QuotaUsage aggregateQuota(Map<RemoteLocation, QuotaUsage> results) {
     long nsCount = 0;
     long ssCount = 0;
     boolean hasQuotaUnSet = false;
 
-    for (Map.Entry<RemoteLocation, Object> entry : results.entrySet()) {
+    for (Map.Entry<RemoteLocation, QuotaUsage> entry : results.entrySet()) {
       RemoteLocation loc = entry.getKey();
-      QuotaUsage usage = (QuotaUsage) entry.getValue();
+      QuotaUsage usage = entry.getValue();
       if (usage != null) {
         // If quota is not set in real FileSystem, the usage
         // value will return -1.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5d6a035/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
index cac3713..4209a49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
@@ -148,6 +148,14 @@ public class RouterRpcClient {
   }
 
   /**
+   * Get the active namenode resolver used by this client.
+   * @return Active namenode resolver.
+   */
+  public ActiveNamenodeResolver getNamenodeResolver() {
+    return this.namenodeResolver;
+  }
+
+  /**
    * Shutdown the client.
    */
   public void shutdown() {
@@ -617,9 +625,9 @@ public class RouterRpcClient {
    * @throws IOException if the success condition is not met, return the first
    *                     remote exception generated.
    */
-  public Object invokeSequential(
+  public <T> T invokeSequential(
       final List<? extends RemoteLocationContext> locations,
-      final RemoteMethod remoteMethod, Class<?> expectedResultClass,
+      final RemoteMethod remoteMethod, Class<T> expectedResultClass,
       Object expectedResultValue) throws IOException {
 
     final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
@@ -639,7 +647,9 @@ public class RouterRpcClient {
         if (isExpectedClass(expectedResultClass, result) &&
             isExpectedValue(expectedResultValue, result)) {
           // Valid result, stop here
-          return result;
+          @SuppressWarnings("unchecked")
+          T ret = (T)result;
+          return ret;
         }
         if (firstResult == null) {
           firstResult = result;
@@ -669,7 +679,9 @@ public class RouterRpcClient {
       throw firstThrownException;
     }
     // Return the last result, whether it is the value we are looking for or a
-    return firstResult;
+    @SuppressWarnings("unchecked")
+    T ret = (T)firstResult;
+    return ret;
   }
 
   /**
@@ -709,6 +721,28 @@ public class RouterRpcClient {
   }
 
   /**
+   * Invoke multiple concurrent proxy calls to different clients. Returns an
+   * array of results.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param <T> The type of the remote location.
+   * @param locations List of remote locations to call concurrently.
+   * @param method The remote method and parameters to invoke.
+   * @param requireResponse If true an exception will be thrown if all calls do
+   *          not complete. If false exceptions are ignored and all data results
+   *          successfully received are returned.
+   * @param standby If the requests should go to the standby namenodes too.
+   * @throws IOException If all the calls throw an exception.
+   */
+  public <T extends RemoteLocationContext, R> void invokeConcurrent(
+      final Collection<T> locations, final RemoteMethod method,
+      boolean requireResponse, boolean standby) throws IOException {
+    invokeConcurrent(locations, method, requireResponse, standby, void.class);
+  }
+
+  /**
    * Invokes multiple concurrent proxy calls to different clients. Returns an
    * array of results.
    *
@@ -716,20 +750,24 @@ public class RouterRpcClient {
    * RemoteException or IOException.
    *
    * @param <T> The type of the remote location.
+   * @param <R> The type of the remote method return.
    * @param locations List of remote locations to call concurrently.
    * @param method The remote method and parameters to invoke.
    * @param requireResponse If true an exception will be thrown if all calls do
    *          not complete. If false exceptions are ignored and all data results
    *          successfully received are returned.
    * @param standby If the requests should go to the standby namenodes too.
+   * @param clazz Type of the remote return type.
    * @return Result of invoking the method per subcluster: nsId -> result.
    * @throws IOException If requiredResponse=true and any of the calls throw an
    *           exception.
    */
-  public <T extends RemoteLocationContext> Map<T, Object> invokeConcurrent(
+  public <T extends RemoteLocationContext, R> Map<T, R> invokeConcurrent(
       final Collection<T> locations, final RemoteMethod method,
-      boolean requireResponse, boolean standby) throws IOException {
-    return invokeConcurrent(locations, method, requireResponse, standby, -1);
+      boolean requireResponse, boolean standby, Class<R> clazz)
+          throws IOException {
+    return invokeConcurrent(
+        locations, method, requireResponse, standby, -1, clazz);
   }
 
   /**
@@ -739,6 +777,8 @@ public class RouterRpcClient {
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
+   * @param <T> The type of the remote location.
+   * @param <R> The type of the remote method return.
    * @param locations List of remote locations to call concurrently.
    * @param method The remote method and parameters to invoke.
    * @param requireResponse If true an exception will be thrown if all calls do
@@ -746,14 +786,15 @@ public class RouterRpcClient {
    *          successfully received are returned.
    * @param standby If the requests should go to the standby namenodes too.
    * @param timeOutMs Timeout for each individual call.
+   * @param clazz Type of the remote return type.
    * @return Result of invoking the method per subcluster: nsId -> result.
    * @throws IOException If requiredResponse=true and any of the calls throw an
    *           exception.
    */
   @SuppressWarnings("unchecked")
-  public <T extends RemoteLocationContext> Map<T, Object> invokeConcurrent(
+  public <T extends RemoteLocationContext, R> Map<T, R> invokeConcurrent(
       final Collection<T> locations, final RemoteMethod method,
-      boolean requireResponse, boolean standby, long timeOutMs)
+      boolean requireResponse, boolean standby, long timeOutMs, Class<R> clazz)
           throws IOException {
 
     final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
@@ -767,7 +808,7 @@ public class RouterRpcClient {
           getNamenodesForNameservice(ns);
       Object[] paramList = method.getParams(location);
       Object result = invokeMethod(ugi, namenodes, m, paramList);
-      return Collections.singletonMap(location, result);
+      return Collections.singletonMap(location, clazz.cast(result));
     }
 
     List<T> orderedLocations = new LinkedList<>();
@@ -817,14 +858,14 @@ public class RouterRpcClient {
       } else {
         futures = executorService.invokeAll(callables);
       }
-      Map<T, Object> results = new TreeMap<>();
+      Map<T, R> results = new TreeMap<>();
       Map<T, IOException> exceptions = new TreeMap<>();
       for (int i=0; i<futures.size(); i++) {
         T location = orderedLocations.get(i);
         try {
           Future<Object> future = futures.get(i);
           Object result = future.get();
-          results.put(location, result);
+          results.put(location, clazz.cast(result));
         } catch (CancellationException ce) {
           T loc = orderedLocations.get(i);
           String msg =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5d6a035/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 73b189e..9afd441 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -28,12 +28,14 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.lang.reflect.Array;
 import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -181,8 +183,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   /** Category of the operation that a thread is executing. */
   private final ThreadLocal<OperationCategory> opCategory = new ThreadLocal<>();
 
+  // Modules implementing groups of RPC calls
   /** Router Quota calls. */
   private final Quota quotaCall;
+  /** Erasure coding calls. */
+  private final ErasureCoding erasureCoding;
+
 
   /**
    * Construct a router RPC server.
@@ -282,6 +288,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
 
     // Initialize modules
     this.quotaCall = new Quota(this.router, this);
+    this.erasureCoding = new ErasureCoding(this);
   }
 
   @Override
@@ -367,7 +374,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    *                          client requests.
    * @throws UnsupportedOperationException If the operation is not supported.
    */
-  private void checkOperation(OperationCategory op, boolean supported)
+  protected void checkOperation(OperationCategory op, boolean supported)
       throws StandbyException, UnsupportedOperationException {
     checkOperation(op);
 
@@ -949,8 +956,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("getListing",
         new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
         new RemoteParam(), startAfter, needLocation);
-    Map<RemoteLocation, Object> listings =
-        rpcClient.invokeConcurrent(locations, method, false, false);
+    Map<RemoteLocation, DirectoryListing> listings =
+        rpcClient.invokeConcurrent(
+            locations, method, false, false, DirectoryListing.class);
 
     Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
     int totalRemainingEntries = 0;
@@ -959,9 +967,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     if (listings != null) {
       // Check the subcluster listing with the smallest name
       String lastName = null;
-      for (Entry<RemoteLocation, Object> entry : listings.entrySet()) {
+      for (Entry<RemoteLocation, DirectoryListing> entry :
+          listings.entrySet()) {
         RemoteLocation location = entry.getKey();
-        DirectoryListing listing = (DirectoryListing) entry.getValue();
+        DirectoryListing listing = entry.getValue();
         if (listing == null) {
           LOG.debug("Cannot get listing from {}", location);
         } else {
@@ -1097,11 +1106,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
 
     RemoteMethod method = new RemoteMethod("getStats");
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, long[]> results =
+        rpcClient.invokeConcurrent(nss, method, true, false, long[].class);
     long[] combinedData = new long[STATS_ARRAY_LENGTH];
-    for (Object o : results.values()) {
-      long[] data = (long[]) o;
+    for (long[] data : results.values()) {
       for (int i = 0; i < combinedData.length && i < data.length; i++) {
         if (data[i] >= 0) {
           combinedData[i] += data[i];
@@ -1134,11 +1142,13 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
         new Class<?>[] {DatanodeReportType.class}, type);
 
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, false, timeOutMs);
-    for (Entry<FederationNamespaceInfo, Object> entry : results.entrySet()) {
+    Map<FederationNamespaceInfo, DatanodeInfo[]> results =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, timeOutMs, DatanodeInfo[].class);
+    for (Entry<FederationNamespaceInfo, DatanodeInfo[]> entry :
+        results.entrySet()) {
       FederationNamespaceInfo ns = entry.getKey();
-      DatanodeInfo[] result = (DatanodeInfo[]) entry.getValue();
+      DatanodeInfo[] result = entry.getValue();
       for (DatanodeInfo node : result) {
         String nodeId = node.getXferAddr();
         if (!datanodesMap.containsKey(nodeId)) {
@@ -1168,10 +1178,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("getDatanodeStorageReport",
         new Class<?>[] {DatanodeReportType.class}, type);
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, false);
-    for (Object r : results.values()) {
-      DatanodeStorageReport[] result = (DatanodeStorageReport[]) r;
+    Map<FederationNamespaceInfo, DatanodeStorageReport[]> results =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, DatanodeStorageReport[].class);
+    for (DatanodeStorageReport[] result : results.values()) {
       for (DatanodeStorageReport node : result) {
         String nodeId = node.getDatanodeInfo().getXferAddr();
         if (!datanodesMap.containsKey(nodeId)) {
@@ -1199,17 +1209,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
         new Class<?>[] {SafeModeAction.class, boolean.class},
         action, isChecked);
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, true);
+    Map<FederationNamespaceInfo, Boolean> results =
+        rpcClient.invokeConcurrent(nss, method, true, true, boolean.class);
 
     // We only report true if all the name space are in safe mode
     int numSafemode = 0;
-    for (Object result : results.values()) {
-      if (result instanceof Boolean) {
-        boolean safemode = (boolean) result;
-        if (safemode) {
-          numSafemode++;
-        }
+    for (boolean safemode : results.values()) {
+      if (safemode) {
+        numSafemode++;
       }
     }
     return numSafemode == results.size();
@@ -1222,18 +1229,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("restoreFailedStorage",
         new Class<?>[] {String.class}, arg);
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
 
     boolean success = true;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Boolean> results =
-        (Map<FederationNamespaceInfo, Boolean>)obj;
-    Collection<Boolean> sucesses = results.values();
-    for (boolean s : sucesses) {
+    for (boolean s : ret.values()) {
       if (!s) {
         success = false;
+        break;
       }
     }
     return success;
@@ -1246,18 +1249,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("saveNamespace",
         new Class<?>[] {Long.class, Long.class}, timeWindow, txGap);
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
 
     boolean success = true;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Boolean> results =
-        (Map<FederationNamespaceInfo, Boolean>)obj;
-    Collection<Boolean> sucesses = results.values();
-    for (boolean s : sucesses) {
+    for (boolean s : ret.values()) {
       if (!s) {
         success = false;
+        break;
       }
     }
     return success;
@@ -1269,17 +1268,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
 
     RemoteMethod method = new RemoteMethod("rollEdits", new Class<?>[] {});
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
 
     // Return the maximum txid
     long txid = 0;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Long> results =
-        (Map<FederationNamespaceInfo, Long>)obj;
-    Collection<Long> txids = results.values();
-    for (long t : txids) {
+    for (long t : ret.values()) {
       if (t > txid) {
         txid = t;
       }
@@ -1314,17 +1308,13 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("rollingUpgrade",
         new Class<?>[] {RollingUpgradeAction.class}, action);
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, RollingUpgradeInfo> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, RollingUpgradeInfo.class);
 
     // Return the first rolling upgrade info
     RollingUpgradeInfo info = null;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, RollingUpgradeInfo> results =
-        (Map<FederationNamespaceInfo, RollingUpgradeInfo>)obj;
-    Collection<RollingUpgradeInfo> infos = results.values();
-    for (RollingUpgradeInfo infoNs : infos) {
+    for (RollingUpgradeInfo infoNs : ret.values()) {
       if (info == null && infoNs != null) {
         info = infoNs;
       }
@@ -1376,10 +1366,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
       final List<RemoteLocation> locations = getLocationsForPath(path, false);
       RemoteMethod method = new RemoteMethod("getContentSummary",
           new Class<?>[] {String.class}, new RemoteParam());
-      @SuppressWarnings("unchecked")
-      Map<String, ContentSummary> results =
-          (Map<String, ContentSummary>) ((Object)rpcClient.invokeConcurrent(
-              locations, method, false, false));
+      Map<RemoteLocation, ContentSummary> results =
+          rpcClient.invokeConcurrent(
+              locations, method, false, false, ContentSummary.class);
       summaries.addAll(results.values());
     } catch (FileNotFoundException e) {
       notFoundException = e;
@@ -1773,17 +1762,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod(
         "getCurrentEditLogTxid", new Class<?>[] {});
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
 
     // Return the maximum txid
     long txid = 0;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Long> results =
-        (Map<FederationNamespaceInfo, Long>)obj;
-    Collection<Long> txids = results.values();
-    for (long t : txids) {
+    for (long t : ret.values()) {
       if (t > txid) {
         txid = t;
       }
@@ -1816,31 +1800,6 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     checkOperation(OperationCategory.WRITE, false);
   }
 
-  @Override
-  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
-      throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
-  }
-
-  @Override // ClientProtocol
-  public ErasureCodingPolicy getErasureCodingPolicy(String src)
-      throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
-  }
-
-  @Override // ClientProtocol
-  public void setErasureCodingPolicy(String src, String ecPolicyName)
-      throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-  }
-
-  @Override // ClientProtocol
-  public void unsetErasureCodingPolicy(String src) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-  }
-
   @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
       StorageType type) throws IOException {
@@ -1894,38 +1853,61 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     return null;
   }
 
-  @Override
+  @Override // ClientProtocol
+  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
+      throws IOException {
+    return erasureCoding.getErasureCodingPolicies();
+  }
+
+  @Override // ClientProtocol
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
+    return erasureCoding.getErasureCodingCodecs();
+  }
+
+  @Override // ClientProtocol
   public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return null;
+    return erasureCoding.addErasureCodingPolicies(policies);
   }
 
-  @Override
-  public void removeErasureCodingPolicy(String arg0) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+  @Override // ClientProtocol
+  public void removeErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.removeErasureCodingPolicy(ecPolicyName);
   }
 
-  @Override
-  public void disableErasureCodingPolicy(String arg0) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+  @Override // ClientProtocol
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.disableErasureCodingPolicy(ecPolicyName);
   }
 
-  @Override
-  public void enableErasureCodingPolicy(String arg0) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+  @Override // ClientProtocol
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.enableErasureCodingPolicy(ecPolicyName);
   }
 
-  @Override
-  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+  @Override // ClientProtocol
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    return erasureCoding.getErasureCodingPolicy(src);
+  }
+
+  @Override // ClientProtocol
+  public void setErasureCodingPolicy(String src, String ecPolicyName)
+      throws IOException {
+    erasureCoding.setErasureCodingPolicy(src, ecPolicyName);
+  }
+
+  @Override // ClientProtocol
+  public void unsetErasureCodingPolicy(String src) throws IOException {
+    erasureCoding.unsetErasureCodingPolicy(src);
   }
 
   @Override
-  public Map<String, String> getErasureCodingCodecs() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
+    return erasureCoding.getECBlockGroupStats();
   }
 
   @Override
@@ -2128,6 +2110,39 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   }
 
   /**
+   * Merge the outputs from multiple namespaces.
+   * @param map Namespace -> Output array.
+   * @param clazz Class of the values.
+   * @return Array with the outputs.
+   */
+  protected static <T> T[] merge(
+      Map<FederationNamespaceInfo, T[]> map, Class<T> clazz) {
+
+    // Put all results into a set to avoid repeats
+    Set<T> ret = new LinkedHashSet<>();
+    for (T[] values : map.values()) {
+      for (T val : values) {
+        ret.add(val);
+      }
+    }
+
+    return toArray(ret, clazz);
+  }
+
+  /**
+   * Convert a set of values into an array.
+   * @param set Input set.
+   * @param clazz Class of the values.
+   * @return Array with the values in set.
+   */
+  private static <T> T[] toArray(Set<T> set, Class<T> clazz) {
+    @SuppressWarnings("unchecked")
+    T[] combinedData = (T[]) Array.newInstance(clazz, set.size());
+    combinedData = set.toArray(combinedData);
+    return combinedData;
+  }
+
+  /**
    * Get quota module implement.
    */
   public Quota getQuotaModule() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5d6a035/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
index 7424499..8f8bd3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
@@ -109,6 +109,8 @@ public class RouterDFSCluster {
   private List<RouterContext> routers;
   /** If the Namenodes are in high availability.*/
   private boolean highAvailability;
+  /** Number of datanodes per nameservice. */
+  private int numDatanodesPerNameservice = 2;
 
   /** Mini cluster. */
   private MiniDFSCluster cluster;
@@ -356,8 +358,8 @@ public class RouterDFSCluster {
         DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
   }
 
-  public RouterDFSCluster(boolean ha, int numNameservices, int numNamnodes) {
-    this(ha, numNameservices, numNamnodes,
+  public RouterDFSCluster(boolean ha, int numNameservices, int numNamenodes) {
+    this(ha, numNameservices, numNamenodes,
         DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
   }
 
@@ -531,6 +533,10 @@ public class RouterDFSCluster {
     }
   }
 
+  public void setNumDatanodesPerNameservice(int num) {
+    this.numDatanodesPerNameservice = num;
+  }
+
   public String getNameservicesKey() {
     StringBuilder sb = new StringBuilder();
     for (String nsId : this.nameservices) {
@@ -658,7 +664,7 @@ public class RouterDFSCluster {
         nnConf.addResource(overrideConf);
       }
       cluster = new MiniDFSCluster.Builder(nnConf)
-          .numDataNodes(nameservices.size()*2)
+          .numDataNodes(nameservices.size() * numDatanodesPerNameservice)
           .nnTopology(topology)
           .build();
       cluster.waitActive();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5d6a035/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
index af506c9..6a8c0e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
@@ -24,19 +24,24 @@ import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.delet
 import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.getFileStatus;
 import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyFileExists;
 import static org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.TEST_STRING;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.net.URISyntaxException;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
@@ -53,10 +58,15 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 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.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -68,12 +78,18 @@ import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service.STATE;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
 
 /**
  * The the RPC interface of the {@link Router} implemented by
@@ -81,6 +97,20 @@ import org.junit.Test;
  */
 public class TestRouterRpc {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRouterRpc.class);
+
+  private static final Comparator<ErasureCodingPolicyInfo> EC_POLICY_CMP =
+      new Comparator<ErasureCodingPolicyInfo>() {
+        public int compare(
+            ErasureCodingPolicyInfo ec0,
+            ErasureCodingPolicyInfo ec1) {
+          String name0 = ec0.getPolicy().getName();
+          String name1 = ec1.getPolicy().getName();
+          return name0.compareTo(name1);
+        }
+      };
+
   /** Federated HDFS cluster. */
   private static RouterDFSCluster cluster;
 
@@ -111,6 +141,8 @@ public class TestRouterRpc {
   @BeforeClass
   public static void globalSetUp() throws Exception {
     cluster = new RouterDFSCluster(false, 2);
+    // We need 6 DNs to test Erasure Coding with RS-6-3-64k
+    cluster.setNumDatanodesPerNameservice(6);
 
     // Start NNs and DNs and wait until ready
     cluster.startCluster();
@@ -144,9 +176,9 @@ public class TestRouterRpc {
     // Wait to ensure NN has fully created its test directories
     Thread.sleep(100);
 
-    // Pick a NS, namenode and router for this test
+    // Default namenode and random router for this test
     this.router = cluster.getRandomRouter();
-    this.ns = cluster.getRandomNameservice();
+    this.ns = cluster.getNameservices().get(0);
     this.namenode = cluster.getNamenode(ns, null);
 
     // Handles to the ClientProtocol interface
@@ -481,7 +513,7 @@ public class TestRouterRpc {
       for (int i = 0; i < data.length; i++) {
         individualData[i] += data[i];
       }
-      assert(data.length == combinedData.length);
+      assertEquals(data.length, combinedData.length);
     }
 
     for (int i = 0; i < combinedData.length && i < individualData.length; i++) {
@@ -489,7 +521,9 @@ public class TestRouterRpc {
         // Skip available storage as this fluctuates in mini cluster
         continue;
       }
-      assertEquals(combinedData[i], individualData[i]);
+      assertEquals("Stats for " + i + " don't match: " +
+          combinedData[i] + "!=" + individualData[i],
+          combinedData[i], individualData[i]);
     }
   }
 
@@ -866,4 +900,153 @@ public class TestRouterRpc {
 
     assertEquals(routerFailure.getClass(), nnFailure.getClass());
   }
+
+  @Test
+  public void testErasureCoding() throws IOException {
+
+    LOG.info("List the available erasurce coding policies");
+    ErasureCodingPolicyInfo[] policies = checkErasureCodingPolicies();
+    for (ErasureCodingPolicyInfo policy : policies) {
+      LOG.info("  {}", policy);
+    }
+
+    LOG.info("List the erasure coding codecs");
+    Map<String, String> codecsRouter = routerProtocol.getErasureCodingCodecs();
+    Map<String, String> codecsNamenode = nnProtocol.getErasureCodingCodecs();
+    assertTrue(Maps.difference(codecsRouter, codecsNamenode).areEqual());
+    for (Entry<String, String> entry : codecsRouter.entrySet()) {
+      LOG.info("  {}: {}", entry.getKey(), entry.getValue());
+    }
+
+    LOG.info("Create a testing directory via the router at the root level");
+    String dirPath = "/testec";
+    String filePath1 = dirPath + "/testfile1";
+    FsPermission permission = new FsPermission("755");
+    routerProtocol.mkdirs(dirPath, permission, false);
+    createFile(routerFS, filePath1, 32);
+    assertTrue(verifyFileExists(routerFS, filePath1));
+    DFSClient file1Protocol = getFileDFSClient(filePath1);
+
+    LOG.info("The policy for the new file should not be set");
+    assertNull(routerProtocol.getErasureCodingPolicy(filePath1));
+    assertNull(file1Protocol.getErasureCodingPolicy(filePath1));
+
+    String policyName = "RS-6-3-1024k";
+    LOG.info("Set policy \"{}\" for \"{}\"", policyName, dirPath);
+    routerProtocol.setErasureCodingPolicy(dirPath, policyName);
+
+    String filePath2 = dirPath + "/testfile2";
+    LOG.info("Create {} in the path with the new EC policy", filePath2);
+    createFile(routerFS, filePath2, 32);
+    assertTrue(verifyFileExists(routerFS, filePath2));
+    DFSClient file2Protocol = getFileDFSClient(filePath2);
+
+    LOG.info("Check that the policy is set for {}", filePath2);
+    ErasureCodingPolicy policyRouter1 =
+        routerProtocol.getErasureCodingPolicy(filePath2);
+    ErasureCodingPolicy policyNamenode1 =
+        file2Protocol.getErasureCodingPolicy(filePath2);
+    assertNotNull(policyRouter1);
+    assertEquals(policyName, policyRouter1.getName());
+    assertEquals(policyName, policyNamenode1.getName());
+
+    LOG.info("Create a new erasure coding policy");
+    String newPolicyName = "RS-6-3-128k";
+    ECSchema ecSchema = new ECSchema(ErasureCodeConstants.RS_CODEC_NAME, 6, 3);
+    ErasureCodingPolicy ecPolicy = new ErasureCodingPolicy(
+        newPolicyName,
+        ecSchema,
+        128 * 1024,
+        (byte) -1);
+    ErasureCodingPolicy[] newPolicies = new ErasureCodingPolicy[] {
+        ecPolicy
+    };
+    AddErasureCodingPolicyResponse[] responses =
+        routerProtocol.addErasureCodingPolicies(newPolicies);
+    assertEquals(1, responses.length);
+    assertTrue(responses[0].isSucceed());
+    routerProtocol.disableErasureCodingPolicy(newPolicyName);
+
+    LOG.info("The new policy should be there and disabled");
+    policies = checkErasureCodingPolicies();
+    boolean found = false;
+    for (ErasureCodingPolicyInfo policy : policies) {
+      LOG.info("  {}" + policy);
+      if (policy.getPolicy().getName().equals(newPolicyName)) {
+        found = true;
+        assertEquals(ErasureCodingPolicyState.DISABLED, policy.getState());
+        break;
+      }
+    }
+    assertTrue(found);
+
+    LOG.info("Set the test folder to use the new policy");
+    routerProtocol.enableErasureCodingPolicy(newPolicyName);
+    routerProtocol.setErasureCodingPolicy(dirPath, newPolicyName);
+
+    LOG.info("Create a file in the path with the new EC policy");
+    String filePath3 = dirPath + "/testfile3";
+    createFile(routerFS, filePath3, 32);
+    assertTrue(verifyFileExists(routerFS, filePath3));
+    DFSClient file3Protocol = getFileDFSClient(filePath3);
+
+    ErasureCodingPolicy policyRouterFile3 =
+        routerProtocol.getErasureCodingPolicy(filePath3);
+    assertEquals(newPolicyName, policyRouterFile3.getName());
+    ErasureCodingPolicy policyNamenodeFile3 =
+        file3Protocol.getErasureCodingPolicy(filePath3);
+    assertEquals(newPolicyName, policyNamenodeFile3.getName());
+
+    LOG.info("Remove the policy and check the one for the test folder");
+    routerProtocol.removeErasureCodingPolicy(newPolicyName);
+    ErasureCodingPolicy policyRouter3 =
+        routerProtocol.getErasureCodingPolicy(filePath3);
+    assertEquals(newPolicyName, policyRouter3.getName());
+    ErasureCodingPolicy policyNamenode3 =
+        file3Protocol.getErasureCodingPolicy(filePath3);
+    assertEquals(newPolicyName, policyNamenode3.getName());
+
+    LOG.info("Check the stats");
+    ECBlockGroupStats statsRouter = routerProtocol.getECBlockGroupStats();
+    ECBlockGroupStats statsNamenode = nnProtocol.getECBlockGroupStats();
+    assertEquals(statsNamenode.toString(), statsRouter.toString());
+  }
+
+  /**
+   * Check the erasure coding policies in the Router and the Namenode.
+   * @return The erasure coding policies.
+   */
+  private ErasureCodingPolicyInfo[] checkErasureCodingPolicies()
+      throws IOException {
+    ErasureCodingPolicyInfo[] policiesRouter =
+        routerProtocol.getErasureCodingPolicies();
+    assertNotNull(policiesRouter);
+    ErasureCodingPolicyInfo[] policiesNamenode =
+        nnProtocol.getErasureCodingPolicies();
+    Arrays.sort(policiesRouter, EC_POLICY_CMP);
+    Arrays.sort(policiesNamenode, EC_POLICY_CMP);
+    assertArrayEquals(policiesRouter, policiesNamenode);
+    return policiesRouter;
+  }
+
+  /**
+   * Find the Namenode for a particular file and return the DFSClient.
+   * @param path Path of the file to check.
+   * @return The DFSClient to the Namenode holding the file.
+   */
+  private DFSClient getFileDFSClient(final String path) {
+    for (String nsId : cluster.getNameservices()) {
+      LOG.info("Checking {} for {}", nsId, path);
+      NamenodeContext nn = cluster.getNamenode(nsId, null);
+      try {
+        DFSClient nnClientProtocol = nn.getClient();
+        if (nnClientProtocol.getFileInfo(path) != null) {
+          return nnClientProtocol;
+        }
+      } catch (Exception ignore) {
+        // ignore
+      }
+    }
+    return null;
+  }
 }
\ No newline at end of file


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