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 vi...@apache.org on 2015/05/25 08:28:08 UTC

hadoop git commit: HDFS-8408. Revisit and refactor ErasureCodingInfo (Contributed by Vinayakumar B)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 6a0afa6b1 -> 1f5f22628


HDFS-8408. Revisit and refactor ErasureCodingInfo (Contributed by Vinayakumar B)


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

Branch: refs/heads/HDFS-7285
Commit: 1f5f226288ba24728ead5198d17e8fde02759408
Parents: 6a0afa6
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon May 25 11:57:17 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Mon May 25 11:57:17 2015 +0530

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  2 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 22 ++-----
 .../hadoop/hdfs/DistributedFileSystem.java      | 16 ++---
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 12 +---
 .../hadoop/hdfs/protocol/ErasureCodingInfo.java | 41 ------------
 .../hdfs/protocol/ErasureCodingZoneInfo.java    | 66 --------------------
 ...tNamenodeProtocolServerSideTranslatorPB.java | 37 +++--------
 .../ClientNamenodeProtocolTranslatorPB.java     | 34 +++-------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 34 ++++------
 .../server/blockmanagement/BlockManager.java    | 12 ++--
 .../namenode/ErasureCodingZoneManager.java      | 10 +--
 .../server/namenode/FSDirStatAndListingOp.java  | 14 ++---
 .../hdfs/server/namenode/FSDirectory.java       |  6 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 23 ++-----
 .../hdfs/server/namenode/NameNodeRpcServer.java | 13 +---
 .../hadoop/hdfs/server/namenode/Namesystem.java | 10 ++-
 .../hdfs/tools/erasurecode/ECCommand.java       | 16 ++---
 .../src/main/proto/ClientNamenodeProtocol.proto |  6 +-
 .../src/main/proto/erasurecoding.proto          | 26 ++------
 .../hadoop/hdfs/TestErasureCodingZones.java     | 16 +++--
 .../test/resources/testErasureCodingConf.xml    | 38 +++++------
 21 files changed, 117 insertions(+), 337 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 4609fb6..d045ee5 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -257,3 +257,5 @@
     HDFS-7768. Change fsck to support EC files.  (Takanobu Asanuma via szetszwo)
 
     HDFS-8382. Remove chunkSize and initialize from erasure coder. (Kai Zheng)
+
+    HDFS-8408. Revisit and refactor ErasureCodingInfo (vinayakumarb)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index b77ae9c..70aec9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -118,8 +118,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.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -3135,19 +3134,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("getErasureCodingInfo", src);
-    try {
-      return namenode.getErasureCodingInfo(src);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-          FileNotFoundException.class, UnresolvedPathException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
   public ECSchema[] getECSchemas() throws IOException {
     checkOpen();
     TraceScope scope = Trace.startSpan("getECSchemas", traceSampler);
@@ -3352,11 +3338,11 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @return Returns the zone information if path is in EC Zone, null otherwise
    * @throws IOException
    */
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+  public ErasureCodingZone getErasureCodingZone(String src) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getErasureCodingZoneInfo", src);
+    TraceScope scope = getPathTraceScope("getErasureCodingZone", src);
     try {
-      return namenode.getErasureCodingZoneInfo(src);
+      return namenode.getErasureCodingZone(src);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(FileNotFoundException.class,
           AccessControlException.class, UnresolvedPathException.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 2e21372..6d55922 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -75,7 +75,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.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -2316,25 +2316,25 @@ public class DistributedFileSystem extends FileSystem {
    * @return Returns the zone information if path is in EC zone, null otherwise
    * @throws IOException
    */
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(final Path path)
+  public ErasureCodingZone getErasureCodingZone(final Path path)
       throws IOException {
     Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<ErasureCodingZoneInfo>() {
+    return new FileSystemLinkResolver<ErasureCodingZone>() {
       @Override
-      public ErasureCodingZoneInfo doCall(final Path p) throws IOException,
+      public ErasureCodingZone doCall(final Path p) throws IOException,
           UnresolvedLinkException {
-        return dfs.getErasureCodingZoneInfo(getPathName(p));
+        return dfs.getErasureCodingZone(getPathName(p));
       }
 
       @Override
-      public ErasureCodingZoneInfo next(final FileSystem fs, final Path p)
+      public ErasureCodingZone next(final FileSystem fs, final Path p)
           throws IOException {
         if (fs instanceof DistributedFileSystem) {
           DistributedFileSystem myDfs = (DistributedFileSystem) fs;
-          return myDfs.getErasureCodingZoneInfo(p);
+          return myDfs.getErasureCodingZone(p);
         }
         throw new UnsupportedOperationException(
-            "Cannot getErasureCodingZoneInfo through a symlink to a "
+            "Cannot getErasureCodingZone through a symlink to a "
                 + "non-DistributedFileSystem: " + path + " -> " + p);
       }
     }.resolve(this, absF);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 4f985ba..d2b8a51 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1467,16 +1467,6 @@ public interface ClientProtocol {
       throws IOException;
 
   /**
-   * Gets the ECInfo for the specified file/directory
-   * 
-   * @param src
-   * @return Returns the ECInfo if the file/directory is erasure coded, null otherwise
-   * @throws IOException
-   */
-  @Idempotent
-  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException;
-
-  /**
    * Gets list of ECSchemas loaded in Namenode
    *
    * @return Returns the list of ECSchemas loaded at Namenode
@@ -1492,5 +1482,5 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException;
+  public ErasureCodingZone getErasureCodingZone(String src) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java
deleted file mode 100644
index bad09b3..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.io.erasurecode.ECSchema;
-
-/**
- * Class to provide information, such as ECSchema, for a file/block.
- */
-public class ErasureCodingInfo {
-  private final String src;
-  private final ECSchema schema;
-
-  public ErasureCodingInfo(String src, ECSchema schema) {
-    this.src = src;
-    this.schema = schema;
-  }
-
-  public String getSrc() {
-    return src;
-  }
-
-  public ECSchema getSchema() {
-    return schema;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java
deleted file mode 100644
index 282eeaf..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.io.erasurecode.ECSchema;
-
-/**
- * Information about the EC Zone at the specified path.
- */
-public class ErasureCodingZoneInfo {
-
-  private String dir;
-  private ECSchema schema;
-  private int cellSize;
-
-  public ErasureCodingZoneInfo(String dir, ECSchema schema, int cellSize) {
-    this.dir = dir;
-    this.schema = schema;
-    this.cellSize = cellSize;
-  }
-
-  /**
-   * Get directory of the EC zone.
-   * 
-   * @return
-   */
-  public String getDir() {
-    return dir;
-  }
-
-  /**
-   * Get the schema for the EC Zone
-   * 
-   * @return
-   */
-  public ECSchema getSchema() {
-    return schema;
-  }
-
-  /**
-   * Get cellSize for the EC Zone
-   */
-  public int getCellSize() {
-    return cellSize;
-  }
-
-  @Override
-  public String toString() {
-    return "Dir: " + getDir() + ", Schema: " + schema + ", cellSize: "
-        + cellSize;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 863b217..4228a65 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
@@ -35,8 +35,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.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -202,10 +201,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptio
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
@@ -1525,22 +1522,6 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController controller,
-      GetErasureCodingInfoRequestProto request) throws ServiceException {
-    try {
-      ErasureCodingInfo ecInfo = server.getErasureCodingInfo(request.getSrc());
-      GetErasureCodingInfoResponseProto.Builder resBuilder = GetErasureCodingInfoResponseProto
-          .newBuilder();
-      if (ecInfo != null) {
-        resBuilder.setECInfo(PBHelper.convertECInfo(ecInfo));
-      }
-      return resBuilder.build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
-  @Override
   public GetECSchemasResponseProto getECSchemas(RpcController controller,
       GetECSchemasRequestProto request) throws ServiceException {
     try {
@@ -1557,13 +1538,13 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetErasureCodingZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller,
-      GetErasureCodingZoneInfoRequestProto request) throws ServiceException {
+  public GetErasureCodingZoneResponseProto getErasureCodingZone(RpcController controller,
+      GetErasureCodingZoneRequestProto request) throws ServiceException {
     try {
-      ErasureCodingZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc());
-      GetErasureCodingZoneInfoResponseProto.Builder builder = GetErasureCodingZoneInfoResponseProto.newBuilder();
-      if (ecZoneInfo != null) {
-        builder.setECZoneInfo(PBHelper.convertECZoneInfo(ecZoneInfo));
+      ErasureCodingZone ecZone = server.getErasureCodingZone(request.getSrc());
+      GetErasureCodingZoneResponseProto.Builder builder = GetErasureCodingZoneResponseProto.newBuilder();
+      if (ecZone != null) {
+        builder.setECZone(PBHelper.convertErasureCodingZone(ecZone));
       }
       return builder.build();
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 336e3a2..2c88a93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -58,8 +58,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.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -168,10 +167,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathR
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
@@ -1553,21 +1550,6 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
-    GetErasureCodingInfoRequestProto req = GetErasureCodingInfoRequestProto.newBuilder()
-        .setSrc(src).build();
-    try {
-      GetErasureCodingInfoResponseProto res = rpcProxy.getErasureCodingInfo(null, req);
-      if (res.hasECInfo()) {
-        return PBHelper.convertECInfo(res.getECInfo());
-      }
-      return null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
   public ECSchema[] getECSchemas() throws IOException {
     try {
       GetECSchemasResponseProto response = rpcProxy.getECSchemas(null,
@@ -1584,14 +1566,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
-    GetErasureCodingZoneInfoRequestProto req = GetErasureCodingZoneInfoRequestProto.newBuilder()
+  public ErasureCodingZone getErasureCodingZone(String src) throws IOException {
+    GetErasureCodingZoneRequestProto req = GetErasureCodingZoneRequestProto.newBuilder()
         .setSrc(src).build();
     try {
-      GetErasureCodingZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo(
+      GetErasureCodingZoneResponseProto response = rpcProxy.getErasureCodingZone(
           null, req);
-      if (response.hasECZoneInfo()) {
-        return PBHelper.convertECZoneInfo(response.getECZoneInfo());
+      if (response.hasECZone()) {
+        return PBHelper.convertErasureCodingZone(response.getECZone());
       }
       return null;
     } catch (ServiceException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 4d0f871..b2415fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -77,13 +77,12 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -135,8 +134,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterComm
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
@@ -203,7 +201,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
@@ -3123,16 +3120,6 @@ public class PBHelper {
         build();
   }
 
-  public static ErasureCodingInfo convertECInfo(ErasureCodingInfoProto ecInfoProto) {
-    return new ErasureCodingInfo(ecInfoProto.getSrc(),
-        convertECSchema(ecInfoProto.getSchema()));
-  }
-
-  public static ErasureCodingInfoProto convertECInfo(ErasureCodingInfo ecInfo) {
-    return ErasureCodingInfoProto.newBuilder().setSrc(ecInfo.getSrc())
-        .setSchema(convertECSchema(ecInfo.getSchema())).build();
-  }
-
   public static ECSchema convertECSchema(ECSchemaProto schema) {
     List<ECSchemaOptionEntryProto> optionsList = schema.getOptionsList();
     Map<String, String> options = new HashMap<>(optionsList.size());
@@ -3157,16 +3144,17 @@ public class PBHelper {
     return builder.build();
   }
 
-  public static ErasureCodingZoneInfoProto convertECZoneInfo(ErasureCodingZoneInfo ecZoneInfo) {
-    return ErasureCodingZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir())
-        .setSchema(convertECSchema(ecZoneInfo.getSchema()))
-        .setCellSize(ecZoneInfo.getCellSize()).build();
+  public static ErasureCodingZoneProto convertErasureCodingZone(
+      ErasureCodingZone ecZone) {
+    return ErasureCodingZoneProto.newBuilder().setDir(ecZone.getDir())
+        .setSchema(convertECSchema(ecZone.getSchema()))
+        .setCellSize(ecZone.getCellSize()).build();
   }
 
-  public static ErasureCodingZoneInfo convertECZoneInfo(ErasureCodingZoneInfoProto ecZoneInfoProto) {
-    return new ErasureCodingZoneInfo(ecZoneInfoProto.getDir(),
-        convertECSchema(ecZoneInfoProto.getSchema()),
-        ecZoneInfoProto.getCellSize());
+  public static ErasureCodingZone convertErasureCodingZone(
+      ErasureCodingZoneProto ecZoneProto) {
+    return new ErasureCodingZone(ecZoneProto.getDir(),
+        convertECSchema(ecZoneProto.getSchema()), ecZoneProto.getCellSize());
   }
   
   public static BlockECRecoveryInfo convertBlockECRecoveryInfo(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index fdcc3c4..d5c31be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -1553,14 +1553,14 @@ public class BlockManager {
             assert rw instanceof ErasureCodingWork;
             assert rw.targets.length > 0;
             String src = block.getBlockCollection().getName();
-            ErasureCodingZoneInfo ecZoneInfo = null;
+            ErasureCodingZone ecZone = null;
             try {
-              ecZoneInfo = namesystem.getErasureCodingZoneInfoForPath(src);
+              ecZone = namesystem.getErasureCodingZoneForPath(src);
             } catch (IOException e) {
               blockLog
-                  .warn("Failed to get the EC zone info for the file {} ", src);
+                  .warn("Failed to get the EC zone for the file {} ", src);
             }
-            if (ecZoneInfo == null) {
+            if (ecZone == null) {
               blockLog.warn("No EC schema found for the file {}. "
                   + "So cannot proceed for recovery", src);
               // TODO: we may have to revisit later for what we can do better to
@@ -1571,7 +1571,7 @@ public class BlockManager {
                 new ExtendedBlock(namesystem.getBlockPoolId(), block),
                 rw.srcNodes, rw.targets,
                 ((ErasureCodingWork) rw).liveBlockIndicies,
-                ecZoneInfo.getSchema(), ecZoneInfo.getCellSize());
+                ecZone.getSchema(), ecZone.getCellSize());
           } else {
             rw.srcNodes[0].addBlockToBeReplicated(block, targets);
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
index 89fecc6..4b3e747 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
@@ -23,7 +23,7 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.XAttrHelper;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -61,11 +61,11 @@ public class ErasureCodingZoneManager {
   }
 
   ECSchema getECSchema(INodesInPath iip) throws IOException {
-    ErasureCodingZoneInfo ecZoneInfo = getECZoneInfo(iip);
-    return ecZoneInfo == null ? null : ecZoneInfo.getSchema();
+    ErasureCodingZone ecZone = getECZone(iip);
+    return ecZone == null ? null : ecZone.getSchema();
   }
 
-  ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+  ErasureCodingZone getECZone(INodesInPath iip) throws IOException {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     List<INode> inodes = iip.getReadOnlyINodes();
@@ -92,7 +92,7 @@ public class ErasureCodingZoneManager {
           String schemaName = WritableUtils.readString(dIn);
           ECSchema schema = dir.getFSNamesystem().getECSchemaManager()
               .getSchema(schemaName);
-          return new ErasureCodingZoneInfo(dir.getInode(inode.getId())
+          return new ErasureCodingZone(dir.getInode(inode.getId())
               .getFullPathName(), schema, cellSize);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index eba5013..1b1d79e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -386,9 +386,9 @@ class FSDirStatAndListingOp {
     final FileEncryptionInfo feInfo = isRawPath ? null :
         fsd.getFileEncryptionInfo(node, snapshot, iip);
     
-    final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip);
-    final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null;
-    final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0;
+    final ErasureCodingZone ecZone = fsd.getECZone(iip);
+    final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
+    final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
 
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
@@ -468,9 +468,9 @@ class FSDirStatAndListingOp {
     }
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
-    final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip);
-    final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null;
-    final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0;
+    final ErasureCodingZone ecZone = fsd.getECZone(iip);
+    final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
+    final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
 
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 9a2b291..879b617 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
@@ -1352,10 +1352,10 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+  ErasureCodingZone getECZone(INodesInPath iip) throws IOException {
     readLock();
     try {
-      return ecZoneManager.getECZoneInfo(iip);
+      return ecZoneManager.getECZone(iip);
     } finally {
       readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index c4549c4..50334a3 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
@@ -182,8 +182,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.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -7823,27 +7822,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
-   * Get the erasure coding information for specified src
-   */
-  ErasureCodingInfo getErasureCodingInfo(String src) throws AccessControlException,
-      UnresolvedLinkException, IOException {
-    ErasureCodingZoneInfo zoneInfo = getErasureCodingZoneInfo(src);
-    if (zoneInfo != null) {
-      return new ErasureCodingInfo(src, zoneInfo.getSchema());
-    }
-    return null;
-  }
-
-  /**
    * Get the erasure coding zone information for specified path
    */
-  ErasureCodingZoneInfo getErasureCodingZoneInfo(String src)
+  ErasureCodingZone getErasureCodingZone(String src)
       throws AccessControlException, UnresolvedLinkException, IOException {
     checkOperation(OperationCategory.READ);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return getErasureCodingZoneInfoForPath(src);
+      return getErasureCodingZoneForPath(src);
     } finally {
       readUnlock();
     }
@@ -8064,7 +8051,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src)
+  public ErasureCodingZone getErasureCodingZoneForPath(String src)
       throws IOException {
     final byte[][] pathComponents = FSDirectory
         .getPathComponentsForReservedPath(src);
@@ -8074,7 +8061,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if (isPermissionEnabled) {
       dir.checkPathAccess(pc, iip, FsAction.READ);
     }
-    return dir.getECZoneInfo(iip);
+    return dir.getECZone(iip);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 9e94b90..1377bbe 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
@@ -84,8 +84,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.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -2036,20 +2035,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
-    checkNNStartup();
-    return namesystem.getErasureCodingInfo(src);
-  }
-
-  @Override // ClientProtocol
   public ECSchema[] getECSchemas() throws IOException {
     checkNNStartup();
     return namesystem.getECSchemas();
   }
 
   @Override // ClientProtocol
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+  public ErasureCodingZone getErasureCodingZone(String src) throws IOException {
     checkNNStartup();
-    return namesystem.getErasureCodingZoneInfo(src);
+    return namesystem.getErasureCodingZone(src);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index a32e800..2a9ab39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -21,11 +21,10 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.security.AccessControlException;
 
@@ -52,13 +51,12 @@ public interface Namesystem extends RwLock, SafeMode {
   public boolean isInSnapshot(BlockCollection bc);
 
   /**
-   * Gets the ECZone info for path
-   * 
+   * Gets the ECZone for path
    * @param src
    *          - path
-   * @return {@link ErasureCodingZoneInfo}
+   * @return {@link ErasureCodingZone}
    * @throws IOException
    */
-  public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src)
+  public ErasureCodingZone getErasureCodingZoneForPath(String src)
       throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
index 2d82208..34965d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFactory;
 import org.apache.hadoop.fs.shell.PathData;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.io.erasurecode.ECSchema;
@@ -47,8 +47,8 @@ public abstract class ECCommand extends Command {
     // Register all commands of Erasure CLI, with a '-' at the beginning in name
     // of the command.
     factory.addClass(CreateECZoneCommand.class, "-" + CreateECZoneCommand.NAME);
-    factory.addClass(GetECZoneInfoCommand.class, "-"
-        + GetECZoneInfoCommand.NAME);
+    factory.addClass(GetECZoneCommand.class, "-"
+        + GetECZoneCommand.NAME);
     factory.addClass(ListECSchemas.class, "-" + ListECSchemas.NAME);
   }
 
@@ -153,8 +153,8 @@ public abstract class ECCommand extends Command {
   /**
    * Get the information about the zone
    */
-  static class GetECZoneInfoCommand extends ECCommand {
-    public static final String NAME = "getZoneInfo";
+  static class GetECZoneCommand extends ECCommand {
+    public static final String NAME = "getZone";
     public static final String USAGE = "<path>";
     public static final String DESCRIPTION =
         "Get information about the EC zone at specified path\n";
@@ -174,9 +174,9 @@ public abstract class ECCommand extends Command {
       super.processPath(item);
       DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
       try {
-        ErasureCodingZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path);
-        if (ecZoneInfo != null) {
-          out.println(ecZoneInfo.toString());
+        ErasureCodingZone ecZone = dfs.getErasureCodingZone(item.path);
+        if (ecZone != null) {
+          out.println(ecZone.toString());
         } else {
           out.println("Path " + item.path + " is not in EC zone");
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
index 0a0a4c4..2b64ca0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -863,10 +863,8 @@ service ClientNamenodeProtocol {
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
       returns(GetEditsFromTxidResponseProto);
-  rpc getErasureCodingInfo(GetErasureCodingInfoRequestProto)
-      returns(GetErasureCodingInfoResponseProto);
   rpc getECSchemas(GetECSchemasRequestProto)
       returns(GetECSchemasResponseProto);
-  rpc getErasureCodingZoneInfo(GetErasureCodingZoneInfoRequestProto) 
-      returns(GetErasureCodingZoneInfoResponseProto);
+  rpc getErasureCodingZone(GetErasureCodingZoneRequestProto)
+      returns(GetErasureCodingZoneResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
index 058ed96..56bb7a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -24,17 +24,9 @@ package hadoop.hdfs;
 import "hdfs.proto";
 
 /**
- * ErasureCodingInfo
+ * ErasureCodingZone
  */
-message ErasureCodingInfoProto {
- required string src = 1;
- required ECSchemaProto schema = 2;
-}
-
-/**
- * ErasureCodingZoneInfo
- */
-message ErasureCodingZoneInfoProto {
+message ErasureCodingZoneProto {
   required string dir = 1;
   required ECSchemaProto schema = 2;
   required uint32 cellSize = 3;
@@ -49,14 +41,6 @@ message CreateErasureCodingZoneRequestProto {
 message CreateErasureCodingZoneResponseProto {
 }
 
-message GetErasureCodingInfoRequestProto {
-  required string src = 1;
-}
-
-message GetErasureCodingInfoResponseProto {
-  optional ErasureCodingInfoProto ECInfo = 1;
-}
-
 message GetECSchemasRequestProto { // void request
 }
 
@@ -64,12 +48,12 @@ message GetECSchemasResponseProto {
   repeated ECSchemaProto schemas = 1;
 }
 
-message GetErasureCodingZoneInfoRequestProto {
+message GetErasureCodingZoneRequestProto {
   required string src = 1; // path to get the zone info
 }
 
-message GetErasureCodingZoneInfoResponseProto {
-  optional ErasureCodingZoneInfoProto ECZoneInfo = 1;
+message GetErasureCodingZoneResponseProto {
+  optional ErasureCodingZoneProto ECZone = 1;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
index 480791e..d724b53 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -177,12 +177,12 @@ public class TestErasureCodingZones {
     final Path ecDir = new Path(src);
     fs.mkdir(ecDir, FsPermission.getDirDefault());
     // dir ECInfo before creating ec zone
-    assertNull(fs.getClient().getErasureCodingInfo(src));
+    assertNull(fs.getClient().getFileInfo(src).getECSchema());
     // dir ECInfo after creating ec zone
     fs.getClient().createErasureCodingZone(src, null, 0); //Default one will be used.
     ECSchema sysDefaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     verifyErasureCodingInfo(src, sysDefaultSchema);
-    fs.create(new Path(ecDir, "/child1")).close();
+    fs.create(new Path(ecDir, "child1")).close();
     // verify for the files in ec zone
     verifyErasureCodingInfo(src + "/child1", sysDefaultSchema);
   }
@@ -198,21 +198,19 @@ public class TestErasureCodingZones {
     final Path ecDir = new Path(src);
     fs.mkdir(ecDir, FsPermission.getDirDefault());
     // dir ECInfo before creating ec zone
-    assertNull(fs.getClient().getErasureCodingInfo(src));
+    assertNull(fs.getClient().getFileInfo(src).getECSchema());
     // dir ECInfo after creating ec zone
     fs.getClient().createErasureCodingZone(src, usingSchema, 0);
     verifyErasureCodingInfo(src, usingSchema);
-    fs.create(new Path(ecDir, "/child1")).close();
+    fs.create(new Path(ecDir, "child1")).close();
     // verify for the files in ec zone
     verifyErasureCodingInfo(src + "/child1", usingSchema);
   }
 
   private void verifyErasureCodingInfo(
       String src, ECSchema usingSchema) throws IOException {
-    ErasureCodingInfo ecInfo = fs.getClient().getErasureCodingInfo(src);
-    assertNotNull("ECInfo should have been non-null", ecInfo);
-    assertEquals(src, ecInfo.getSrc());
-    ECSchema schema = ecInfo.getSchema();
+    HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src);
+    ECSchema schema = hdfsFileStatus.getECSchema();
     assertNotNull(schema);
     assertEquals("Actually used schema should be equal with target schema",
         usingSchema, schema);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f5f2262/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index 66892f0..ee1a19a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -67,9 +67,9 @@
     </test>
 
     <test>
-      <description>help: getZoneInfo command</description>
+      <description>help: getZone command</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -help getZoneInfo</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -help getZone</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
@@ -80,7 +80,7 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-getZoneInfo &lt;path&gt;(.)*</expected-output>
+          <expected-output>^-getZone &lt;path&gt;(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -145,7 +145,7 @@
       <test-commands>
         <command>-fs NAMENODE -mkdir /eczone</command>
         <ec-admin-command>-fs NAMENODE -createZone /eczone</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone /eczone</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /eczone</command>
@@ -159,10 +159,10 @@
     </test>
 
     <test>
-      <description>getZoneInfo : get information about the EC zone at specified path not in zone</description>
+      <description>getZone : get information about the EC zone at specified path not in zone</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /noec</command>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo /noec</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone /noec</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /noec</command>
@@ -176,11 +176,11 @@
     </test>
 
     <test>
-      <description>getZoneInfo : get information about the EC zone at specified path</description>
+      <description>getZone : get information about the EC zone at specified path</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /eczone</command>
         <ec-admin-command>-fs NAMENODE -createZone -s RS-6-3 /eczone</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone /eczone</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /eczone</command>
@@ -194,12 +194,12 @@
     </test>
 
     <test>
-      <description>getZoneInfo : get EC zone at specified file path</description>
+      <description>getZone : get EC zone at specified file path</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /eczone</command>
         <ec-admin-command>-fs NAMENODE -createZone -s RS-6-3 /eczone</ec-admin-command>
         <command>-fs NAMENODE -touchz /eczone/ecfile</command>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone/ecfile</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone /eczone/ecfile</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rm /eczone/ecfile</command>
@@ -313,24 +313,24 @@
     </test>
 
     <test>
-      <description>getZoneInfo : illegal parameters - path is missing</description>
+      <description>getZone : illegal parameters - path is missing</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo </ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone </ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-getZoneInfo: &lt;path&gt; is missing(.)*</expected-output>
+          <expected-output>^-getZone: &lt;path&gt; is missing(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZoneInfo : illegal parameters - too many arguments</description>
+      <description>getZone : illegal parameters - too many arguments</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone /eczone /eczone</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rm /eczone</command>
@@ -338,22 +338,22 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>-getZoneInfo: Too many arguments</expected-output>
+          <expected-output>-getZone: Too many arguments</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZoneInfo : illegal parameters - no such file</description>
+      <description>getZone : illegal parameters - no such file</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone /eczone</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^getZoneInfo: `/eczone': No such file or directory(.)*</expected-output>
+          <expected-output>^getZone: `/eczone': No such file or directory(.)*</expected-output>
         </comparator>
       </comparators>
     </test>