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 ji...@apache.org on 2015/05/17 01:59:04 UTC

[36/50] hadoop git commit: HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding. Contributed by Uma Maheswara Rao G

HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-7285
Commit: fa2834467ca1e70f480fbdb860d6860da0b9fcfd
Parents: be86bf5
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Thu May 7 16:26:01 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Sat May 16 15:16:06 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  10 +-
 .../hadoop/hdfs/DFSStripedInputStream.java      |   2 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |  10 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   4 +-
 .../org/apache/hadoop/hdfs/protocol/ECInfo.java |  41 ------
 .../apache/hadoop/hdfs/protocol/ECZoneInfo.java |  56 --------
 .../hadoop/hdfs/protocol/ErasureCodingInfo.java |  41 ++++++
 .../hdfs/protocol/ErasureCodingZoneInfo.java    |  56 ++++++++
 ...tNamenodeProtocolServerSideTranslatorPB.java |  18 +--
 .../ClientNamenodeProtocolTranslatorPB.java     |  16 +--
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  24 ++--
 .../hdfs/server/namenode/ECSchemaManager.java   | 127 -------------------
 .../namenode/ErasureCodingSchemaManager.java    | 127 +++++++++++++++++++
 .../namenode/ErasureCodingZoneManager.java      |  12 +-
 .../hdfs/server/namenode/FSDirectory.java       |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  24 ++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |   8 +-
 .../hdfs/tools/erasurecode/ECCommand.java       |   4 +-
 .../src/main/proto/ClientNamenodeProtocol.proto |   4 +-
 .../src/main/proto/erasurecoding.proto          |  16 +--
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |   8 +-
 .../org/apache/hadoop/hdfs/TestECSchemas.java   |   2 +-
 .../hadoop/hdfs/TestErasureCodingZones.java     |  10 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |  10 +-
 .../server/namenode/TestStripedINodeFile.java   |  16 +--
 26 files changed, 328 insertions(+), 325 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 8729f8a..11e8376 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -183,3 +183,6 @@
 
     HDFS-8334. Erasure coding: rename DFSStripedInputStream related test 
     classes. (Zhe Zhang)
+
+    HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding
+    (umamahesh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 63c27ef..71fdc34 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,8 @@ 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.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -1191,7 +1191,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     //    Get block info from namenode
     TraceScope scope = getPathTraceScope("newDFSInputStream", src);
     try {
-      ECInfo info = getErasureCodingInfo(src);
+      ErasureCodingInfo info = getErasureCodingInfo(src);
       if (info != null) {
         return new DFSStripedInputStream(this, src, verifyChecksum, info);
       } else {
@@ -3132,7 +3132,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public ECInfo getErasureCodingInfo(String src) throws IOException {
+  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
     checkOpen();
     TraceScope scope = getPathTraceScope("getErasureCodingInfo", src);
     try {
@@ -3349,7 +3349,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @return Returns the zone information if path is in EC Zone, null otherwise
    * @throws IOException
    */
-  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
     checkOpen();
     TraceScope scope = getPathTraceScope("getErasureCodingZoneInfo", src);
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
index 13c4743..7cb7b6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -127,7 +127,7 @@ public class DFSStripedInputStream extends DFSInputStream {
   private final CompletionService<Integer> readingService;
 
   DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
-      ECInfo ecInfo) throws IOException {
+      ErasureCodingInfo ecInfo) throws IOException {
     super(dfsClient, src, verifyChecksum);
     // ECInfo is restored from NN just before reading striped file.
     assert ecInfo != null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 ede4f48..723dbcd 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
@@ -72,7 +72,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.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -2306,18 +2306,18 @@ public class DistributedFileSystem extends FileSystem {
    * @return Returns the zone information if path is in EC zone, null otherwise
    * @throws IOException
    */
-  public ECZoneInfo getErasureCodingZoneInfo(final Path path)
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(final Path path)
       throws IOException {
     Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<ECZoneInfo>() {
+    return new FileSystemLinkResolver<ErasureCodingZoneInfo>() {
       @Override
-      public ECZoneInfo doCall(final Path p) throws IOException,
+      public ErasureCodingZoneInfo doCall(final Path p) throws IOException,
           UnresolvedLinkException {
         return dfs.getErasureCodingZoneInfo(getPathName(p));
       }
 
       @Override
-      public ECZoneInfo next(final FileSystem fs, final Path p)
+      public ErasureCodingZoneInfo next(final FileSystem fs, final Path p)
           throws IOException {
         if (fs instanceof DistributedFileSystem) {
           DistributedFileSystem myDfs = (DistributedFileSystem) fs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 76e2d12..b0b457c 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
@@ -1474,7 +1474,7 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public ECInfo getErasureCodingInfo(String src) throws IOException;
+  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException;
 
   /**
    * Gets list of ECSchemas loaded in Namenode
@@ -1492,5 +1492,5 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException;
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java
deleted file mode 100644
index ca642c2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.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 ECInfo {
-  private final String src;
-  private final ECSchema schema;
-
-  public ECInfo(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/fa283446/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java
deleted file mode 100644
index ecfb92e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java
+++ /dev/null
@@ -1,56 +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 ECZoneInfo {
-
-  private String dir;
-  private ECSchema schema;
-
-  public ECZoneInfo(String dir, ECSchema schema) {
-    this.dir = dir;
-    this.schema = schema;
-  }
-
-  /**
-   * 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;
-  }
-
-  @Override
-  public String toString() {
-    return "Dir: " + getDir() + ", Schema: " + schema;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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
new file mode 100644
index 0000000..bad09b3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java
@@ -0,0 +1,41 @@
+/**
+ * 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/fa283446/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
new file mode 100644
index 0000000..ec0efbd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java
@@ -0,0 +1,56 @@
+/**
+ * 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;
+
+  public ErasureCodingZoneInfo(String dir, ECSchema schema) {
+    this.dir = dir;
+    this.schema = schema;
+  }
+
+  /**
+   * 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;
+  }
+
+  @Override
+  public String toString() {
+    return "Dir: " + getDir() + ", Schema: " + schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 f9bdb71..bae753b 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,8 @@ 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.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -202,8 +202,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.GetECZoneInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto;
+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.CreateErasureCodingZoneRequestProto;
@@ -1527,7 +1527,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController controller,
       GetErasureCodingInfoRequestProto request) throws ServiceException {
     try {
-      ECInfo ecInfo = server.getErasureCodingInfo(request.getSrc());
+      ErasureCodingInfo ecInfo = server.getErasureCodingInfo(request.getSrc());
       GetErasureCodingInfoResponseProto.Builder resBuilder = GetErasureCodingInfoResponseProto
           .newBuilder();
       if (ecInfo != null) {
@@ -1556,11 +1556,11 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetECZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller,
-      GetECZoneInfoRequestProto request) throws ServiceException {
+  public GetErasureCodingZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller,
+      GetErasureCodingZoneInfoRequestProto request) throws ServiceException {
     try {
-      ECZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc());
-      GetECZoneInfoResponseProto.Builder builder = GetECZoneInfoResponseProto.newBuilder();
+      ErasureCodingZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc());
+      GetErasureCodingZoneInfoResponseProto.Builder builder = GetErasureCodingZoneInfoResponseProto.newBuilder();
       if (ecZoneInfo != null) {
         builder.setECZoneInfo(PBHelper.convertECZoneInfo(ecZoneInfo));
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 6c1c971..014fcef 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,8 @@ 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.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -168,8 +168,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.GetECZoneInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto;
+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.CreateErasureCodingZoneRequestProto;
@@ -1550,7 +1550,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ECInfo getErasureCodingInfo(String src) throws IOException {
+  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
     GetErasureCodingInfoRequestProto req = GetErasureCodingInfoRequestProto.newBuilder()
         .setSrc(src).build();
     try {
@@ -1581,11 +1581,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
-    GetECZoneInfoRequestProto req = GetECZoneInfoRequestProto.newBuilder()
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+    GetErasureCodingZoneInfoRequestProto req = GetErasureCodingZoneInfoRequestProto.newBuilder()
         .setSrc(src).build();
     try {
-      GetECZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo(
+      GetErasureCodingZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo(
           null, req);
       if (response.hasECZoneInfo()) {
         return PBHelper.convertECZoneInfo(response.getECZoneInfo());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 e230232..26bdf34 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,13 @@ 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.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 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.ECInfo;
+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,10 +135,10 @@ 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.ECInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECZoneInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
@@ -3117,13 +3117,13 @@ public class PBHelper {
         build();
   }
 
-  public static ECInfo convertECInfo(ECInfoProto ecInfoProto) {
-    return new ECInfo(ecInfoProto.getSrc(),
+  public static ErasureCodingInfo convertECInfo(ErasureCodingInfoProto ecInfoProto) {
+    return new ErasureCodingInfo(ecInfoProto.getSrc(),
         convertECSchema(ecInfoProto.getSchema()));
   }
 
-  public static ECInfoProto convertECInfo(ECInfo ecInfo) {
-    return ECInfoProto.newBuilder().setSrc(ecInfo.getSrc())
+  public static ErasureCodingInfoProto convertECInfo(ErasureCodingInfo ecInfo) {
+    return ErasureCodingInfoProto.newBuilder().setSrc(ecInfo.getSrc())
         .setSchema(convertECSchema(ecInfo.getSchema())).build();
   }
 
@@ -3151,13 +3151,13 @@ public class PBHelper {
     return builder.build();
   }
 
-  public static ECZoneInfoProto convertECZoneInfo(ECZoneInfo ecZoneInfo) {
-    return ECZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir())
+  public static ErasureCodingZoneInfoProto convertECZoneInfo(ErasureCodingZoneInfo ecZoneInfo) {
+    return ErasureCodingZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir())
         .setSchema(convertECSchema(ecZoneInfo.getSchema())).build();
   }
 
-  public static ECZoneInfo convertECZoneInfo(ECZoneInfoProto ecZoneInfoProto) {
-    return new ECZoneInfo(ecZoneInfoProto.getDir(),
+  public static ErasureCodingZoneInfo convertECZoneInfo(ErasureCodingZoneInfoProto ecZoneInfoProto) {
+    return new ErasureCodingZoneInfo(ecZoneInfoProto.getDir(),
         convertECSchema(ecZoneInfoProto.getSchema()));
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
deleted file mode 100644
index 2d63498..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
+++ /dev/null
@@ -1,127 +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.server.namenode;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-/**
- * This manages EC schemas predefined and activated in the system.
- * It loads customized schemas and syncs with persisted ones in
- * NameNode image.
- *
- * This class is instantiated by the FSNamesystem.
- */
-@InterfaceAudience.LimitedPrivate({"HDFS"})
-public final class ECSchemaManager {
-
-  /**
-   * TODO: HDFS-8095
-   */
-  private static final int DEFAULT_DATA_BLOCKS = 6;
-  private static final int DEFAULT_PARITY_BLOCKS = 3;
-  private static final String DEFAULT_CODEC_NAME = "rs";
-  private static final String DEFAULT_SCHEMA_NAME = "RS-6-3";
-  private static final ECSchema SYS_DEFAULT_SCHEMA =
-      new ECSchema(DEFAULT_SCHEMA_NAME,
-               DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
-
-  //We may add more later.
-  private static ECSchema[] SYS_SCHEMAS = new ECSchema[] {
-      SYS_DEFAULT_SCHEMA
-  };
-
-  /**
-   * All active EC activeSchemas maintained in NN memory for fast querying,
-   * identified and sorted by its name.
-   */
-  private final Map<String, ECSchema> activeSchemas;
-
-  ECSchemaManager() {
-
-    this.activeSchemas = new TreeMap<String, ECSchema>();
-    for (ECSchema schema : SYS_SCHEMAS) {
-      activeSchemas.put(schema.getSchemaName(), schema);
-    }
-
-    /**
-     * TODO: HDFS-7859 persist into NameNode
-     * load persistent schemas from image and editlog, which is done only once
-     * during NameNode startup. This can be done here or in a separate method.
-     */
-  }
-
-  /**
-   * Get system defined schemas.
-   * @return system schemas
-   */
-  public static ECSchema[] getSystemSchemas() {
-    return SYS_SCHEMAS;
-  }
-
-  /**
-   * Get system-wide default EC schema, which can be used by default when no
-   * schema is specified for an EC zone.
-   * @return schema
-   */
-  public static ECSchema getSystemDefaultSchema() {
-    return SYS_DEFAULT_SCHEMA;
-  }
-
-  /**
-   * Tell the specified schema is the system default one or not.
-   * @param schema
-   * @return true if it's the default false otherwise
-   */
-  public static boolean isSystemDefault(ECSchema schema) {
-    if (schema == null) {
-      throw new IllegalArgumentException("Invalid schema parameter");
-    }
-
-    // schema name is the identifier.
-    return SYS_DEFAULT_SCHEMA.getSchemaName().equals(schema.getSchemaName());
-  }
-
-  /**
-   * Get all EC schemas that's available to use.
-   * @return all EC schemas
-   */
-  public ECSchema[] getSchemas() {
-    ECSchema[] results = new ECSchema[activeSchemas.size()];
-    return activeSchemas.values().toArray(results);
-  }
-
-  /**
-   * Get the EC schema specified by the schema name.
-   * @param schemaName
-   * @return EC schema specified by the schema name
-   */
-  public ECSchema getSchema(String schemaName) {
-    return activeSchemas.get(schemaName);
-  }
-
-  /**
-   * Clear and clean up
-   */
-  public void clear() {
-    activeSchemas.clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java
new file mode 100644
index 0000000..4c4aae9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java
@@ -0,0 +1,127 @@
+/**
+ * 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.namenode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * This manages EC schemas predefined and activated in the system.
+ * It loads customized schemas and syncs with persisted ones in
+ * NameNode image.
+ *
+ * This class is instantiated by the FSNamesystem.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+public final class ErasureCodingSchemaManager {
+
+  /**
+   * TODO: HDFS-8095
+   */
+  private static final int DEFAULT_DATA_BLOCKS = 6;
+  private static final int DEFAULT_PARITY_BLOCKS = 3;
+  private static final String DEFAULT_CODEC_NAME = "rs";
+  private static final String DEFAULT_SCHEMA_NAME = "RS-6-3";
+  private static final ECSchema SYS_DEFAULT_SCHEMA =
+      new ECSchema(DEFAULT_SCHEMA_NAME,
+               DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
+
+  //We may add more later.
+  private static ECSchema[] SYS_SCHEMAS = new ECSchema[] {
+      SYS_DEFAULT_SCHEMA
+  };
+
+  /**
+   * All active EC activeSchemas maintained in NN memory for fast querying,
+   * identified and sorted by its name.
+   */
+  private final Map<String, ECSchema> activeSchemas;
+
+  ErasureCodingSchemaManager() {
+
+    this.activeSchemas = new TreeMap<String, ECSchema>();
+    for (ECSchema schema : SYS_SCHEMAS) {
+      activeSchemas.put(schema.getSchemaName(), schema);
+    }
+
+    /**
+     * TODO: HDFS-7859 persist into NameNode
+     * load persistent schemas from image and editlog, which is done only once
+     * during NameNode startup. This can be done here or in a separate method.
+     */
+  }
+
+  /**
+   * Get system defined schemas.
+   * @return system schemas
+   */
+  public static ECSchema[] getSystemSchemas() {
+    return SYS_SCHEMAS;
+  }
+
+  /**
+   * Get system-wide default EC schema, which can be used by default when no
+   * schema is specified for an EC zone.
+   * @return schema
+   */
+  public static ECSchema getSystemDefaultSchema() {
+    return SYS_DEFAULT_SCHEMA;
+  }
+
+  /**
+   * Tell the specified schema is the system default one or not.
+   * @param schema
+   * @return true if it's the default false otherwise
+   */
+  public static boolean isSystemDefault(ECSchema schema) {
+    if (schema == null) {
+      throw new IllegalArgumentException("Invalid schema parameter");
+    }
+
+    // schema name is the identifier.
+    return SYS_DEFAULT_SCHEMA.getSchemaName().equals(schema.getSchemaName());
+  }
+
+  /**
+   * Get all EC schemas that's available to use.
+   * @return all EC schemas
+   */
+  public ECSchema[] getSchemas() {
+    ECSchema[] results = new ECSchema[activeSchemas.size()];
+    return activeSchemas.values().toArray(results);
+  }
+
+  /**
+   * Get the EC schema specified by the schema name.
+   * @param schemaName
+   * @return EC schema specified by the schema name
+   */
+  public ECSchema getSchema(String schemaName) {
+    return activeSchemas.get(schemaName);
+  }
+
+  /**
+   * Clear and clean up
+   */
+  public void clear() {
+    activeSchemas.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 14d4e29..6b6add7 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
@@ -22,7 +22,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.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import java.io.IOException;
@@ -53,11 +53,11 @@ public class ErasureCodingZoneManager {
   }
 
   ECSchema getECSchema(INodesInPath iip) throws IOException {
-    ECZoneInfo ecZoneInfo = getECZoneInfo(iip);
+    ErasureCodingZoneInfo ecZoneInfo = getECZoneInfo(iip);
     return ecZoneInfo == null ? null : ecZoneInfo.getSchema();
   }
 
-  ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+  ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     List<INode> inodes = iip.getReadOnlyINodes();
@@ -79,9 +79,9 @@ public class ErasureCodingZoneManager {
       for (XAttr xAttr : xAttrs) {
         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
           String schemaName = new String(xAttr.getValue());
-          ECSchema schema = dir.getFSNamesystem().getSchemaManager()
+          ECSchema schema = dir.getFSNamesystem().getECSchemaManager()
               .getSchema(schemaName);
-          return new ECZoneInfo(inode.getFullPathName(), schema);
+          return new ErasureCodingZoneInfo(inode.getFullPathName(), schema);
         }
       }
     }
@@ -110,7 +110,7 @@ public class ErasureCodingZoneManager {
 
     // System default schema will be used since no specified.
     if (schema == null) {
-      schema = ECSchemaManager.getSystemDefaultSchema();
+      schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     }
 
     // Now persist the schema name in xattr

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 9c92f64..3f619ff 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.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
@@ -1349,7 +1349,7 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+  ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
     readLock();
     try {
       return ecZoneManager.getECZoneInfo(iip);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 c955f33..d476482 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
@@ -181,8 +181,8 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 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.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -430,7 +430,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final BlockManager blockManager;
   private final SnapshotManager snapshotManager;
   private final CacheManager cacheManager;
-  private final ECSchemaManager schemaManager;
+  private final ErasureCodingSchemaManager ecSchemaManager;
   private final DatanodeStatistics datanodeStatistics;
 
   private String nameserviceId;
@@ -610,7 +610,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     leaseManager.removeAllLeases();
     snapshotManager.clearSnapshottableDirs();
     cacheManager.clear();
-    schemaManager.clear();
+    ecSchemaManager.clear();
     setImageLoaded(false);
     blockManager.clear();
   }
@@ -850,7 +850,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       this.dir = new FSDirectory(this, conf);
       this.snapshotManager = new SnapshotManager(dir);
       this.cacheManager = new CacheManager(this, conf, blockManager);
-      this.schemaManager = new ECSchemaManager();
+      this.ecSchemaManager = new ErasureCodingSchemaManager();
       this.safeMode = new SafeModeInfo(conf);
       this.topConf = new TopConf(conf);
       this.auditLoggers = initAuditLoggers(conf);
@@ -6815,8 +6815,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /** @return the schema manager. */
-  public ECSchemaManager getSchemaManager() {
-    return schemaManager;
+  public ErasureCodingSchemaManager getECSchemaManager() {
+    return ecSchemaManager;
   }
 
   @Override  // NameNodeMXBean
@@ -7762,11 +7762,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get the erasure coding information for specified src
    */
-  ECInfo getErasureCodingInfo(String src) throws AccessControlException,
+  ErasureCodingInfo getErasureCodingInfo(String src) throws AccessControlException,
       UnresolvedLinkException, IOException {
     ECSchema schema = getECSchemaForPath(src);
     if (schema != null) {
-      return new ECInfo(src, schema);
+      return new ErasureCodingInfo(src, schema);
     }
     return null;
   }
@@ -7774,7 +7774,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get the erasure coding zone information for specified path
    */
-  ECZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException,
+  ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException,
       UnresolvedLinkException, IOException {
     checkOperation(OperationCategory.READ);
     final byte[][] pathComponents = FSDirectory
@@ -7803,7 +7803,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return schemaManager.getSchemas();
+      return ecSchemaManager.getSchemas();
     } finally {
       readUnlock();
     }
@@ -7818,7 +7818,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return schemaManager.getSchema(schemaName);
+      return ecSchemaManager.getSchema(schemaName);
     } finally {
       readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 dcf0607..5a69b2f 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,8 @@ 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.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -2035,7 +2035,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public ECInfo getErasureCodingInfo(String src) throws IOException {
+  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
     checkNNStartup();
     return namesystem.getErasureCodingInfo(src);
   }
@@ -2047,7 +2047,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
     checkNNStartup();
     return namesystem.getErasureCodingZoneInfo(src);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 802a46d..d53844d 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.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.StringUtils;
@@ -164,7 +164,7 @@ public abstract class ECCommand extends Command {
       super.processPath(item);
       DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
       try {
-        ECZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path);
+        ErasureCodingZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path);
         out.println(ecZoneInfo.toString());
       } catch (IOException e) {
         throw new IOException("Unable to create EC zone for the path "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 046120d..0a0a4c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -867,6 +867,6 @@ service ClientNamenodeProtocol {
       returns(GetErasureCodingInfoResponseProto);
   rpc getECSchemas(GetECSchemasRequestProto)
       returns(GetECSchemasResponseProto);
-  rpc getErasureCodingZoneInfo(GetECZoneInfoRequestProto) 
-      returns(GetECZoneInfoResponseProto);
+  rpc getErasureCodingZoneInfo(GetErasureCodingZoneInfoRequestProto) 
+      returns(GetErasureCodingZoneInfoResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 702f6fd..7a19a80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -43,17 +43,17 @@ message ECSchemaProto {
 }
 
 /**
- * ECInfo
+ * ErasureCodingInfo
  */
-message ECInfoProto {
+message ErasureCodingInfoProto {
  required string src = 1;
  required ECSchemaProto schema = 2;
 }
 
 /**
- * ECZoneInfo
+ * ErasureCodingZoneInfo
  */
-message ECZoneInfoProto {
+message ErasureCodingZoneInfoProto {
   required string dir = 1;
   required ECSchemaProto schema = 2;
 }
@@ -71,7 +71,7 @@ message GetErasureCodingInfoRequestProto {
 }
 
 message GetErasureCodingInfoResponseProto {
-  optional ECInfoProto ECInfo = 1;
+  optional ErasureCodingInfoProto ECInfo = 1;
 }
 
 message GetECSchemasRequestProto { // void request
@@ -81,12 +81,12 @@ message GetECSchemasResponseProto {
   repeated ECSchemaProto schemas = 1;
 }
 
-message GetECZoneInfoRequestProto {
+message GetErasureCodingZoneInfoRequestProto {
   required string src = 1; // path to get the zone info
 }
 
-message GetECZoneInfoResponseProto {
-  optional ECZoneInfoProto ECZoneInfo = 1;
+message GetErasureCodingZoneInfoResponseProto {
+  optional ErasureCodingZoneInfoProto ECZoneInfo = 1;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
index a1f704d..4da9c26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
@@ -24,7 +24,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -34,7 +34,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
-import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -54,8 +54,8 @@ public class TestDFSStripedInputStream {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
-  private ECInfo info = new ECInfo(filePath.toString(),
-      ECSchemaManager.getSystemDefaultSchema());
+  private ErasureCodingInfo info = new ErasureCodingInfo(filePath.toString(),
+      ErasureCodingSchemaManager.getSystemDefaultSchema());
   private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
   private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
   private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
index 83d208a..88198c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
@@ -22,7 +22,7 @@ import static org.junit.Assert.*;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.After;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/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 f1aec82..59818c0 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
@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.ECInfo;
-import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.io.erasurecode.ECSchema;
@@ -158,7 +158,7 @@ public class TestErasureCodingZones {
     assertNull(fs.getClient().getErasureCodingInfo(src));
     // dir ECInfo after creating ec zone
     fs.getClient().createErasureCodingZone(src, null); //Default one will be used.
-    ECSchema sysDefaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema sysDefaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     verifyErasureCodingInfo(src, sysDefaultSchema);
     fs.create(new Path(ecDir, "/child1")).close();
     // verify for the files in ec zone
@@ -167,7 +167,7 @@ public class TestErasureCodingZones {
 
   @Test
   public void testGetErasureCodingInfo() throws Exception {
-    ECSchema[] sysSchemas = ECSchemaManager.getSystemSchemas();
+    ECSchema[] sysSchemas = ErasureCodingSchemaManager.getSystemSchemas();
     assertTrue("System schemas should be of only 1 for now",
         sysSchemas.length == 1);
 
@@ -187,7 +187,7 @@ public class TestErasureCodingZones {
 
   private void verifyErasureCodingInfo(
       String src, ECSchema usingSchema) throws IOException {
-    ECInfo ecInfo = fs.getClient().getErasureCodingInfo(src);
+    ErasureCodingInfo ecInfo = fs.getClient().getErasureCodingInfo(src);
     assertNotNull("ECInfo should have been non-null", ecInfo);
     assertEquals(src, ecInfo.getSrc());
     ECSchema schema = ecInfo.getSchema();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
index f580cbb..1be00b6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@ -71,7 +71,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
@@ -663,7 +663,7 @@ public class TestPBHelper {
     short[] liveBlkIndices0 = new short[2];
     BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo(
         new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0,
-        liveBlkIndices0, ECSchemaManager.getSystemDefaultSchema());
+        liveBlkIndices0, ErasureCodingSchemaManager.getSystemDefaultSchema());
     DatanodeInfo[] dnInfos1 = new DatanodeInfo[] {
         DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
     DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil
@@ -677,7 +677,7 @@ public class TestPBHelper {
     short[] liveBlkIndices1 = new short[2];
     BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo(
         new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1,
-        liveBlkIndices1, ECSchemaManager.getSystemDefaultSchema());
+        liveBlkIndices1, ErasureCodingSchemaManager.getSystemDefaultSchema());
     List<BlockECRecoveryInfo> blkRecoveryInfosList = new ArrayList<BlockECRecoveryInfo>();
     blkRecoveryInfosList.add(blkECRecoveryInfo0);
     blkRecoveryInfosList.add(blkECRecoveryInfo1);
@@ -723,8 +723,8 @@ public class TestPBHelper {
     ECSchema ecSchema2 = blkECRecoveryInfo2.getECSchema();
     // Compare ECSchemas same as default ECSchema as we used system default
     // ECSchema used in this test
-    compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema1);
-    compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema2);
+    compareECSchemas(ErasureCodingSchemaManager.getSystemDefaultSchema(), ecSchema1);
+    compareECSchemas(ErasureCodingSchemaManager.getSystemDefaultSchema(), ecSchema2);
   }
 
   private void compareECSchemas(ECSchema ecSchema1, ECSchema ecSchema2) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa283446/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
index f97943b..7a330b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@ -60,7 +60,7 @@ public class TestStripedINodeFile {
 
   @Test
   public void testBlockStripedTotalBlockCount() {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk,
@@ -72,7 +72,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedLength()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -87,7 +87,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -116,7 +116,7 @@ public class TestStripedINodeFile {
   @Test
   public void testMultipleBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk1 = new Block(1);
@@ -141,7 +141,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedFileSize()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -160,7 +160,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedUCFileSize()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -177,7 +177,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedComputeQuotaUsage()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -204,7 +204,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedUCComputeQuotaUsage()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);