http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index 3c9adc4..f6a5ece 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@ -69,8 +69,7 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
@@ -267,10 +266,10 @@ public final class ErasureCodingWorker {
         new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL);
 
     ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
-      ECSchema schema = recoveryInfo.getECSchema();
-      dataBlkNum = schema.getNumDataUnits();
-      parityBlkNum = schema.getNumParityUnits();
-      cellSize = recoveryInfo.getCellSize();
+      ErasureCodingPolicy ecPolicy = recoveryInfo.getErasureCodingPolicy();
+      dataBlkNum = ecPolicy.getNumDataUnits();
+      parityBlkNum = ecPolicy.getNumParityUnits();
+      cellSize = ecPolicy.getCellSize();
 
       blockGroup = recoveryInfo.getExtendedBlock();
       final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 
1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index dcab075..dc6acd5 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
@@ -177,7 +177,7 @@ public class Mover {
   }
 
   DBlock newDBlock(LocatedBlock lb, List<MLocation> locations,
-                   ECSchema ecSchema) {
+                   ErasureCodingPolicy ecPolicy) {
     Block blk = lb.getBlock().getLocalBlock();
     DBlock db;
     if (lb.isStriped()) {
@@ -186,7 +186,7 @@ public class Mover {
       for (int i = 0; i < indices.length; i++) {
         indices[i] = (byte) lsb.getBlockIndices()[i];
       }
-      db = new DBlockStriped(blk, indices, (short) ecSchema.getNumDataUnits());
+      db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits());
     } else {
       db = new DBlock(blk);
     }
@@ -373,7 +373,7 @@ public class Mover {
       List<StorageType> types = policy.chooseStorageTypes(
           status.getReplication());
 
-      final ECSchema ecSchema = status.getECSchema();
+      final ErasureCodingPolicy ecPolicy = status.getErasureCodingPolicy();
       final LocatedBlocks locatedBlocks = status.getBlockLocations();
       boolean hasRemaining = false;
       final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
@@ -390,7 +390,7 @@ public class Mover {
         final StorageTypeDiff diff = new StorageTypeDiff(types,
             lb.getStorageTypes());
         if (!diff.removeOverlap(true)) {
-          if (scheduleMoves4Block(diff, lb, ecSchema)) {
+          if (scheduleMoves4Block(diff, lb, ecPolicy)) {
             hasRemaining |= (diff.existing.size() > 1 &&
                 diff.expected.size() > 1);
           }
@@ -400,12 +400,12 @@ public class Mover {
     }
 
     boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb,
-                                ECSchema ecSchema) {
+                                ErasureCodingPolicy ecPolicy) {
       final List<MLocation> locations = MLocation.toLocations(lb);
       if (!(lb instanceof LocatedStripedBlock)) {
         Collections.shuffle(locations);
       }
-      final DBlock db = newDBlock(lb, locations, ecSchema);
+      final DBlock db = newDBlock(lb, locations, ecPolicy);
 
       for (final StorageType t : diff.existing) {
         for (final MLocation ml : locations) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
new file mode 100644
index 0000000..71ac36a
--- /dev/null
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -0,0 +1,115 @@
+/**
+ * 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.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * This manages erasure coding policies predefined and activated in the system.
+ * It loads customized policies and syncs with persisted ones in
+ * NameNode image.
+ *
+ * This class is instantiated by the FSNamesystem.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+public final class ErasureCodingPolicyManager {
+
+  /**
+   * TODO: HDFS-8095
+   */
+  private static final int DEFAULT_DATA_BLOCKS = 6;
+  private static final int DEFAULT_PARITY_BLOCKS = 3;
+  private static final int DEFAULT_CELLSIZE = 64 * 1024;
+  private static final String DEFAULT_CODEC_NAME = "rs";
+  private static final String DEFAULT_POLICY_NAME = "RS-6-3-64k";
+  private static final ECSchema SYS_DEFAULT_SCHEMA = new ECSchema(
+      DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
+  private static final ErasureCodingPolicy SYS_DEFAULT_POLICY =
+      new ErasureCodingPolicy(DEFAULT_POLICY_NAME, SYS_DEFAULT_SCHEMA,
+      DEFAULT_CELLSIZE);
+
+  //We may add more later.
+  private static ErasureCodingPolicy[] SYS_POLICY = new ErasureCodingPolicy[] {
+      SYS_DEFAULT_POLICY
+  };
+
+  /**
+   * All active policies maintained in NN memory for fast querying,
+   * identified and sorted by its name.
+   */
+  private final Map<String, ErasureCodingPolicy> activePolicies;
+
+  ErasureCodingPolicyManager() {
+
+    this.activePolicies = new TreeMap<>();
+    for (ErasureCodingPolicy policy : SYS_POLICY) {
+      activePolicies.put(policy.getName(), policy);
+    }
+
+    /**
+     * TODO: HDFS-7859 persist into NameNode
+     * load persistent policies 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 policies.
+   * @return system policies
+   */
+  public static ErasureCodingPolicy[] getSystemPolices() {
+    return SYS_POLICY;
+  }
+
+  /**
+   * Get system-wide default policy, which can be used by default
+   * when no policy is specified for an EC zone.
+   * @return ecPolicy
+   */
+  public static ErasureCodingPolicy getSystemDefaultPolicy() {
+    return SYS_DEFAULT_POLICY;
+  }
+
+  /**
+   * Get all policies that's available to use.
+   * @return all policies
+   */
+  public ErasureCodingPolicy[] getPolicies() {
+    ErasureCodingPolicy[] results = new 
ErasureCodingPolicy[activePolicies.size()];
+    return activePolicies.values().toArray(results);
+  }
+
+  /**
+   * Get the policy specified by the policy name.
+   */
+  public ErasureCodingPolicy getPolicy(String name) {
+    return activePolicies.get(name);
+  }
+
+  /**
+   * Clear and clean up
+   */
+  public void clear() {
+    activePolicies.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/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
deleted file mode 100644
index 4c4aae9..0000000
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.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 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/1d37a881/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 2638126..22d821f 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,11 +23,10 @@ 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.ErasureCodingPolicy;
 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;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -60,9 +59,9 @@ public class ErasureCodingZoneManager {
     this.dir = dir;
   }
 
-  ECSchema getErasureCodingSchema(INodesInPath iip) throws IOException {
+  ErasureCodingPolicy getErasureCodingPolicy(INodesInPath iip) throws 
IOException {
     ErasureCodingZone ecZone = getErasureCodingZone(iip);
-    return ecZone == null ? null : ecZone.getSchema();
+    return ecZone == null ? null : ecZone.getErasureCodingPolicy();
   }
 
   ErasureCodingZone getErasureCodingZone(INodesInPath iip) throws IOException {
@@ -88,12 +87,11 @@ public class ErasureCodingZoneManager {
         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) 
{
           ByteArrayInputStream bIn=new ByteArrayInputStream(xAttr.getValue());
           DataInputStream dIn=new DataInputStream(bIn);
-          int cellSize = WritableUtils.readVInt(dIn);
-          String schemaName = WritableUtils.readString(dIn);
-          ECSchema schema = dir.getFSNamesystem()
-              .getErasureCodingSchemaManager().getSchema(schemaName);
+          String ecPolicyName = WritableUtils.readString(dIn);
+          ErasureCodingPolicy ecPolicy = dir.getFSNamesystem()
+              .getErasureCodingPolicyManager().getPolicy(ecPolicyName);
           return new ErasureCodingZone(dir.getInode(inode.getId())
-              .getFullPathName(), schema, cellSize);
+              .getFullPathName(), ecPolicy);
         }
       }
     }
@@ -101,7 +99,7 @@ public class ErasureCodingZoneManager {
   }
 
   List<XAttr> createErasureCodingZone(final INodesInPath srcIIP,
-      ECSchema schema, int cellSize) throws IOException {
+      ErasureCodingPolicy ecPolicy) throws IOException {
     assert dir.hasWriteLock();
     Preconditions.checkNotNull(srcIIP, "INodes cannot be null");
     String src = srcIIP.getPath();
@@ -115,29 +113,22 @@ public class ErasureCodingZoneManager {
       throw new IOException("Attempt to create an erasure coding zone " +
           "for a file " + src);
     }
-    if (getErasureCodingSchema(srcIIP) != null) {
+    if (getErasureCodingPolicy(srcIIP) != null) {
       throw new IOException("Directory " + src + " is already in an " +
           "erasure coding zone.");
     }
 
-    // System default schema will be used since no specified.
-    if (schema == null) {
-      schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
+    // System default erasure coding policy will be used since no specified.
+    if (ecPolicy == null) {
+      ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
     }
 
-    if (cellSize <= 0) {
-      cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-    }
-
-    // Write the cellsize first and then schema name
     final XAttr ecXAttr;
     DataOutputStream dOut = null;
     try {
       ByteArrayOutputStream bOut = new ByteArrayOutputStream();
       dOut = new DataOutputStream(bOut);
-      WritableUtils.writeVInt(dOut, cellSize);
-      // Now persist the schema name in xattr
-      WritableUtils.writeString(dOut, schema.getSchemaName());
+      WritableUtils.writeString(dOut, ecPolicy.getName());
       ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE,
           bOut.toByteArray());
     } finally {
@@ -158,10 +149,12 @@ public class ErasureCodingZoneManager {
     if (srcZone != null && srcZone.getDir().equals(src) && dstZone == null) {
       return;
     }
-    final ECSchema srcSchema = (srcZone != null) ? srcZone.getSchema() : null;
-    final ECSchema dstSchema = (dstZone != null) ? dstZone.getSchema() : null;
-    if ((srcSchema != null && !srcSchema.equals(dstSchema)) ||
-        (dstSchema != null && !dstSchema.equals(srcSchema))) {
+    final ErasureCodingPolicy srcECPolicy =
+        srcZone != null ? srcZone.getErasureCodingPolicy() : null;
+    final ErasureCodingPolicy dstECPolicy =
+        dstZone != null ? dstZone.getErasureCodingPolicy() : null;
+    if (srcECPolicy != null && !srcECPolicy.equals(dstECPolicy) ||
+        dstECPolicy != null && !dstECPolicy.equals(srcECPolicy)) {
       throw new IOException(
           src + " can't be moved because the source and destination have " +
               "different erasure coding policies.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index fd7ef33..8c515d1 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 
 /**
  * Helper class to perform erasure coding related operations.
@@ -43,15 +43,14 @@ final class FSDirErasureCodingOp {
    * @param fsn namespace
    * @param srcArg the path of a directory which will be the root of the
    *          erasure coding zone. The directory must be empty.
-   * @param schema ECSchema for the erasure coding zone
-   * @param cellSize Cell size of stripe
+   * @param ecPolicy erasure coding policy for the erasure coding zone
    * @param logRetryCache whether to record RPC ids in editlog for retry
    *          cache rebuilding
    * @return {@link HdfsFileStatus}
    * @throws IOException
    */
   static HdfsFileStatus createErasureCodingZone(final FSNamesystem fsn,
-      final String srcArg, final ECSchema schema, final int cellSize,
+      final String srcArg, final ErasureCodingPolicy ecPolicy,
       final boolean logRetryCache) throws IOException {
     assert fsn.hasWriteLock();
 
@@ -68,7 +67,7 @@ final class FSDirErasureCodingOp {
     try {
       iip = fsd.getINodesInPath4Write(src, false);
       xAttrs = fsn.getErasureCodingZoneManager().createErasureCodingZone(
-          iip, schema, cellSize);
+          iip, ecPolicy);
     } finally {
       fsd.writeUnlock();
     }
@@ -120,7 +119,7 @@ final class FSDirErasureCodingOp {
     assert fsn.hasReadLock();
 
     final INodesInPath iip = getINodesInPath(fsn, srcArg);
-    return getErasureCodingSchemaForPath(fsn, iip) != null;
+    return getErasureCodingPolicyForPath(fsn, iip) != null;
   }
 
   /**
@@ -133,49 +132,35 @@ final class FSDirErasureCodingOp {
    */
   static boolean isInErasureCodingZone(final FSNamesystem fsn,
       final INodesInPath iip) throws IOException {
-    return getErasureCodingSchema(fsn, iip) != null;
+    return getErasureCodingPolicy(fsn, iip) != null;
   }
 
   /**
-   * Get erasure coding schema.
+   * Get the erasure coding policy.
    *
    * @param fsn namespace
    * @param iip inodes in the path containing the file
-   * @return {@link ECSchema}
+   * @return {@link ErasureCodingPolicy}
    * @throws IOException
    */
-  static ECSchema getErasureCodingSchema(final FSNamesystem fsn,
+  static ErasureCodingPolicy getErasureCodingPolicy(final FSNamesystem fsn,
       final INodesInPath iip) throws IOException {
     assert fsn.hasReadLock();
 
-    return getErasureCodingSchemaForPath(fsn, iip);
+    return getErasureCodingPolicyForPath(fsn, iip);
   }
 
   /**
-   * Get available erasure coding schemas.
+   * Get available erasure coding polices.
    *
    * @param fsn namespace
-   * @return {@link ECSchema} array
+   * @return {@link ErasureCodingPolicy} array
    */
-  static ECSchema[] getErasureCodingSchemas(final FSNamesystem fsn)
+  static ErasureCodingPolicy[] getErasureCodingPolicies(final FSNamesystem fsn)
       throws IOException {
     assert fsn.hasReadLock();
 
-    return fsn.getErasureCodingSchemaManager().getSchemas();
-  }
-
-  /**
-   * Get the ECSchema specified by the name.
-   *
-   * @param fsn namespace
-   * @param schemaName schema name
-   * @return {@link ECSchema}
-   */
-  static ECSchema getErasureCodingSchema(final FSNamesystem fsn,
-      final String schemaName) throws IOException {
-    assert fsn.hasReadLock();
-
-    return fsn.getErasureCodingSchemaManager().getSchema(schemaName);
+    return fsn.getErasureCodingPolicyManager().getPolicies();
   }
 
   private static INodesInPath getINodesInPath(final FSNamesystem fsn,
@@ -204,12 +189,12 @@ final class FSDirErasureCodingOp {
     }
   }
 
-  private static ECSchema getErasureCodingSchemaForPath(final FSNamesystem fsn,
+  private static ErasureCodingPolicy getErasureCodingPolicyForPath(final 
FSNamesystem fsn,
       final INodesInPath iip) throws IOException {
     final FSDirectory fsd = fsn.getFSDirectory();
     fsd.readLock();
     try {
-      return fsn.getErasureCodingZoneManager().getErasureCodingSchema(iip);
+      return fsn.getErasureCodingZoneManager().getErasureCodingPolicy(iip);
     } finally {
       fsd.readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/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 1e4ee83..bad7c42 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,6 +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.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -40,7 +41,6 @@ import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import 
org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -318,7 +318,7 @@ class FSDirStatAndListingOp {
       if (fsd.getINode4DotSnapshot(srcs) != null) {
         return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
             HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
-            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null, 0);
+            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
       }
       return null;
     }
@@ -388,8 +388,8 @@ class FSDirStatAndListingOp {
 
     final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
         fsd.getFSNamesystem(), iip);
-    final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
-    final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
+    final ErasureCodingPolicy ecPolicy =
+        ecZone != null ? ecZone.getErasureCodingPolicy() : null;
 
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
@@ -421,8 +421,7 @@ class FSDirStatAndListingOp {
         childrenNum,
         feInfo,
         storagePolicy,
-        schema,
-        cellSize);
+        ecPolicy);
   }
 
   private static INodeAttributes getINodeAttributes(
@@ -471,8 +470,8 @@ class FSDirStatAndListingOp {
     }
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
-    final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
-    final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
+    final ErasureCodingPolicy ecPolicy =
+        ecZone != null ? ecZone.getErasureCodingPolicy() : null;
 
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
@@ -481,8 +480,7 @@ class FSDirStatAndListingOp {
           getPermissionForFileStatus(nodeAttrs, isEncrypted),
           nodeAttrs.getUserName(), nodeAttrs.getGroupName(),
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
-          node.getId(), loc, childrenNum, feInfo, storagePolicy, schema,
-          cellSize);
+          node.getId(), loc, childrenNum, feInfo, storagePolicy, ecPolicy);
     // Set caching information for the located blocks.
     if (loc != null) {
       CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 254e850..8f4f51a 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -51,7 +52,6 @@ import 
org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.util.ChunkedArrayList;
@@ -532,16 +532,15 @@ class FSDirWriteFileOp {
       if (isStriped) {
         ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
             fsd.getFSNamesystem(), inodesInPath);
-        ECSchema ecSchema = ecZone.getSchema();
-        short numDataUnits = (short) ecSchema.getNumDataUnits();
-        short numParityUnits = (short) ecSchema.getNumParityUnits();
+        ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy();
+        short numDataUnits = (short) ecPolicy.getNumDataUnits();
+        short numParityUnits = (short) ecPolicy.getNumParityUnits();
         short numLocations = (short) (numDataUnits + numParityUnits);
 
         // check quota limits and updated space consumed
         fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
             numLocations, true);
-        blockInfo = new BlockInfoUnderConstructionStriped(block, ecSchema,
-            ecZone.getCellSize(),
+        blockInfo = new BlockInfoUnderConstructionStriped(block, ecPolicy,
             HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
       } else {
         // check quota limits and updated space consumed

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index a1f38a3..b5b7178 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -992,7 +992,7 @@ public class FSEditLogLoader {
     boolean isStriped = ecZone != null;
     if (isStriped) {
       newBlockInfo = new BlockInfoUnderConstructionStriped(newBlock,
-          ecZone.getSchema(), ecZone.getCellSize());
+          ecZone.getErasureCodingPolicy());
     } else {
       newBlockInfo = new BlockInfoUnderConstructionContiguous(newBlock,
           file.getPreferredBlockReplication());
@@ -1078,7 +1078,7 @@ public class FSEditLogLoader {
           // until several blocks in?
           if (isStriped) {
             newBI = new BlockInfoUnderConstructionStriped(newBlock,
-                ecZone.getSchema(), ecZone.getCellSize());
+                ecZone.getErasureCodingPolicy());
           } else {
             newBI = new BlockInfoUnderConstructionContiguous(newBlock,
                 file.getPreferredBlockReplication());
@@ -1088,11 +1088,9 @@ public class FSEditLogLoader {
           // is only executed when loading edits written by prior
           // versions of Hadoop. Current versions always log
           // OP_ADD operations as each block is allocated.
-          // TODO: ECSchema can be restored from persisted file (HDFS-7859).
           if (isStriped) {
             newBI = new BlockInfoStriped(newBlock,
-                ErasureCodingSchemaManager.getSystemDefaultSchema(),
-                ecZone.getCellSize());
+                ErasureCodingPolicyManager.getSystemDefaultPolicy());
           } else {
             newBI = new BlockInfoContiguous(newBlock,
                 file.getPreferredBlockReplication());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index 3efb933..653bd4a 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
@@ -66,7 +67,6 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.protobuf.ByteString;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 
 @InterfaceAudience.Private
 public final class FSImageFormatPBINode {
@@ -327,17 +327,13 @@ public final class FSImageFormatPBINode {
       short replication = (short) f.getReplication();
       boolean isStriped = f.getIsStriped();
       LoaderContext state = parent.getLoaderContext();
-      ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
+      ErasureCodingPolicy ecPolicy = 
ErasureCodingPolicyManager.getSystemDefaultPolicy();
 
-      if (isStriped) {
-        Preconditions.checkState(f.hasStripingCellSize());
-      }
       BlockInfo[] blocks = new BlockInfo[bp.size()];
       for (int i = 0; i < bp.size(); ++i) {
         BlockProto b = bp.get(i);
         if (isStriped) {
-          blocks[i] = new BlockInfoStriped(PBHelper.convert(b), schema,
-              (int)f.getStripingCellSize());
+          blocks[i] = new BlockInfoStriped(PBHelper.convert(b), ecPolicy);
         } else {
           blocks[i] = new BlockInfoContiguous(PBHelper.convert(b),
               replication);
@@ -373,8 +369,7 @@ public final class FSImageFormatPBINode {
           final BlockInfo ucBlk;
           if (isStriped) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
-            ucBlk = new BlockInfoUnderConstructionStriped(striped,
-                schema, (int)f.getStripingCellSize());
+            ucBlk = new BlockInfoUnderConstructionStriped(striped, ecPolicy);
           } else {
             ucBlk = new BlockInfoUnderConstructionContiguous(lastBlk,
                 replication);
@@ -656,16 +651,6 @@ public final class FSImageFormatPBINode {
         }
       }
 
-      if (n.isStriped()) {
-        if (blocks != null && blocks.length > 0) {
-          BlockInfo firstBlock = blocks[0];
-          Preconditions.checkState(firstBlock.isStriped());
-          b.setStripingCellSize(((BlockInfoStriped)firstBlock).getCellSize());
-        } else {
-          b.setStripingCellSize(HdfsConstants.BLOCK_STRIPED_CELL_SIZE);
-        }
-      }
-
       FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();
       if (uc != null) {
         INodeSection.FileUnderConstructionFeature f =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/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 541bfa4..657f29d 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
@@ -180,6 +180,7 @@ 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.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -262,7 +263,6 @@ import 
org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.ipc.RetryCache;
 import org.apache.hadoop.ipc.Server;
@@ -426,7 +426,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   private final BlockManager blockManager;
   private final SnapshotManager snapshotManager;
   private final CacheManager cacheManager;
-  private final ErasureCodingSchemaManager ecSchemaManager;
+  private final ErasureCodingPolicyManager ecPolicyManager;
   private final DatanodeStatistics datanodeStatistics;
 
   private String nameserviceId;
@@ -606,7 +606,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     leaseManager.removeAllLeases();
     snapshotManager.clearSnapshottableDirs();
     cacheManager.clear();
-    ecSchemaManager.clear();
+    ecPolicyManager.clear();
     setImageLoaded(false);
     blockManager.clear();
   }
@@ -846,7 +846,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.ecSchemaManager = new ErasureCodingSchemaManager();
+      this.ecPolicyManager = new ErasureCodingPolicyManager();
       this.safeMode = new SafeModeInfo(conf);
       this.topConf = new TopConf(conf);
       this.auditLoggers = initAuditLoggers(conf);
@@ -3679,16 +3679,16 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     if (fileINode.isStriped()) {
       final ErasureCodingZone ecZone = FSDirErasureCodingOp
           .getErasureCodingZone(this, iip);
-      final ECSchema ecSchema = ecZone.getSchema();
-      final short numDataUnits = (short) ecSchema.getNumDataUnits();
-      final short numParityUnits = (short) ecSchema.getNumParityUnits();
+      final ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy();
+      final short numDataUnits = (short) ecPolicy.getNumDataUnits();
+      final short numParityUnits = (short) ecPolicy.getNumParityUnits();
 
       final long numBlocks = numDataUnits + numParityUnits;
       final long fullBlockGroupSize =
           fileINode.getPreferredBlockSize() * numBlocks;
 
       final BlockInfoStriped striped = new BlockInfoStriped(commitBlock,
-          ecSchema, ecZone.getCellSize());
+          ecPolicy);
       final long actualBlockGroupSize = striped.spaceConsumed();
 
       diff = fullBlockGroupSize - actualBlockGroupSize;
@@ -6676,9 +6676,9 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     return cacheManager;
   }
 
-  /** @return the ErasureCodingSchemaManager. */
-  public ErasureCodingSchemaManager getErasureCodingSchemaManager() {
-    return ecSchemaManager;
+  /** @return the ErasureCodingPolicyManager. */
+  public ErasureCodingPolicyManager getErasureCodingPolicyManager() {
+    return ecPolicyManager;
   }
 
   /** @return the ErasureCodingZoneManager. */
@@ -7581,14 +7581,13 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
    * Create an erasure coding zone on directory src.
    * @param srcArg  the path of a directory which will be the root of the
    *                erasure coding zone. The directory must be empty.
-   * @param schema  ECSchema for the erasure coding zone
-   * @param cellSize Cell size of stripe 
+   * @param ecPolicy  erasure coding policy for the erasure coding zone
    * @throws AccessControlException  if the caller is not the superuser.
    * @throws UnresolvedLinkException if the path can't be resolved.
    * @throws SafeModeException       if the Namenode is in safe mode.
    */
-  void createErasureCodingZone(final String srcArg, final ECSchema schema,
-      int cellSize, final boolean logRetryCache) throws IOException,
+  void createErasureCodingZone(final String srcArg, final ErasureCodingPolicy
+      ecPolicy, final boolean logRetryCache) throws IOException,
       UnresolvedLinkException, SafeModeException, AccessControlException {
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.WRITE);
@@ -7599,7 +7598,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create erasure coding zone on " + srcArg);
       resultingStat = FSDirErasureCodingOp.createErasureCodingZone(this,
-          srcArg, schema, cellSize, logRetryCache);
+          srcArg, ecPolicy, logRetryCache);
       success = true;
     } finally {
       writeUnlock();
@@ -7627,30 +7626,15 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   }
 
   /**
-   * Get available erasure coding schemas
+   * Get available erasure coding polices
    */
-  ECSchema[] getErasureCodingSchemas() throws IOException {
+  ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
     checkOperation(OperationCategory.READ);
     waitForLoadingFSImage();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return FSDirErasureCodingOp.getErasureCodingSchemas(this);
-    } finally {
-      readUnlock();
-    }
-  }
-
-  /**
-   * Get the ECSchema specified by the name
-   */
-  ECSchema getErasureCodingSchema(String schemaName) throws IOException {
-    checkOperation(OperationCategory.READ);
-    waitForLoadingFSImage();
-    readLock();
-    try {
-      checkOperation(OperationCategory.READ);
-      return FSDirErasureCodingOp.getErasureCodingSchema(this, schemaName);
+      return FSDirErasureCodingOp.getErasureCodingPolicies(this);
     } finally {
       readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/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 031ae08..fffa0f0 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,6 +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.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -143,7 +144,6 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RetryCache;
@@ -1823,7 +1823,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public void createErasureCodingZone(String src, ECSchema schema, int 
cellSize)
+  public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy)
       throws IOException {
     checkNNStartup();
     final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
@@ -1832,8 +1832,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
     boolean success = false;
     try {
-      namesystem.createErasureCodingZone(src, schema, cellSize,
-          cacheEntry != null);
+      namesystem.createErasureCodingZone(src, ecPolicy, cacheEntry != null);
       success = true;
     } finally {
       RetryCache.setState(cacheEntry, success);
@@ -2035,9 +2034,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public ECSchema[] getECSchemas() throws IOException {
+  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
     checkNNStartup();
-    return namesystem.getErasureCodingSchemas();
+    return namesystem.getErasureCodingPolicies();
   }
 
   @Override // ClientProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 3e4a287..3a2c3d4 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -572,7 +572,7 @@ public class NamenodeFsck implements 
DataEncryptionKeyFactory {
 
       // count expected replicas
       short targetFileReplication;
-      if (file.getECSchema() != null) {
+      if (file.getErasureCodingPolicy() != null) {
         assert storedBlock instanceof BlockInfoStriped;
         targetFileReplication = ((BlockInfoStriped) storedBlock)
             .getRealTotalBlockNum();
@@ -1158,11 +1158,11 @@ public class NamenodeFsck implements 
DataEncryptionKeyFactory {
 
   @VisibleForTesting
   static class ErasureCodingResult extends Result {
-    final String defaultSchema;
+    final String defaultECPolicy;
 
     ErasureCodingResult(Configuration conf) {
-      defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema()
-          .getSchemaName();
+      defaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy()
+          .getName();
     }
 
     @Override
@@ -1239,7 +1239,7 @@ public class NamenodeFsck implements 
DataEncryptionKeyFactory {
             ((float) (numMisReplicatedBlocks * 100) / (float) totalBlocks))
             .append(" %)");
       }
-      res.append("\n Default schema:\t\t").append(defaultSchema)
+      res.append("\n Default ecPolicy:\t\t").append(defaultECPolicy)
           .append("\n Average block group size:\t").append(
           getReplicationFactor()).append("\n Missing block 
groups:\t\t").append(
           missingIds.size()).append("\n Corrupt block groups:\t\t").append(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
index 56a1546..55ae7b6 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -77,31 +77,28 @@ public class BlockECRecoveryCommand extends DatanodeCommand 
{
     private String[] targetStorageIDs;
     private StorageType[] targetStorageTypes;
     private final short[] liveBlockIndices;
-    private final ECSchema ecSchema;
-    private final int cellSize;
+    private final ErasureCodingPolicy ecPolicy;
 
     public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
         DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices,
-        ECSchema ecSchema, int cellSize) {
+        ErasureCodingPolicy ecPolicy) {
       this(block, sources, DatanodeStorageInfo
           .toDatanodeInfos(targetDnStorageInfo), DatanodeStorageInfo
           .toStorageIDs(targetDnStorageInfo), DatanodeStorageInfo
-          .toStorageTypes(targetDnStorageInfo), liveBlockIndices, ecSchema,
-          cellSize);
+          .toStorageTypes(targetDnStorageInfo), liveBlockIndices, ecPolicy);
     }
 
     public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
         DatanodeInfo[] targets, String[] targetStorageIDs,
         StorageType[] targetStorageTypes, short[] liveBlockIndices,
-        ECSchema ecSchema, int cellSize) {
+        ErasureCodingPolicy ecPolicy) {
       this.block = block;
       this.sources = sources;
       this.targets = targets;
       this.targetStorageIDs = targetStorageIDs;
       this.targetStorageTypes = targetStorageTypes;
       this.liveBlockIndices = liveBlockIndices;
-      this.ecSchema = ecSchema;
-      this.cellSize = cellSize;
+      this.ecPolicy = ecPolicy;
     }
 
     public ExtendedBlock getExtendedBlock() {
@@ -128,12 +125,8 @@ public class BlockECRecoveryCommand extends 
DatanodeCommand {
       return liveBlockIndices;
     }
     
-    public ECSchema getECSchema() {
-      return ecSchema;
-    }
-
-    public int getCellSize() {
-      return cellSize;
+    public ErasureCodingPolicy getErasureCodingPolicy() {
+      return ecPolicy;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/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 03026d8..f3260da 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
@@ -31,9 +31,8 @@ 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.ErasureCodingZone;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
-import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -49,7 +48,7 @@ public abstract class ECCommand extends Command {
     factory.addClass(CreateECZoneCommand.class, "-" + 
CreateECZoneCommand.NAME);
     factory.addClass(GetECZoneCommand.class, "-"
         + GetECZoneCommand.NAME);
-    factory.addClass(ListECSchemas.class, "-" + ListECSchemas.NAME);
+    factory.addClass(ListPolicies.class, "-" + ListPolicies.NAME);
   }
 
   @Override
@@ -77,35 +76,24 @@ public abstract class ECCommand extends Command {
   }
 
   /**
-   * Create EC encoding zone command. Zones are created to use specific EC
-   * encoding schema, other than default while encoding the files under some
-   * specific directory.
+   * A command to create an EC zone for a path, with a erasure coding policy 
name.
    */
   static class CreateECZoneCommand extends ECCommand {
     public static final String NAME = "createZone";
-    public static final String USAGE = "[-s <schemaName>] [-c <cellSize>] 
<path>";
+    public static final String USAGE = "[-s <policyName>] <path>";
     public static final String DESCRIPTION = 
-        "Create a zone to encode files using a specified schema\n"
+        "Create a zone to encode files using a specified policy\n"
         + "Options :\n"
-        + "  -s <schemaName> : EC schema name to encode files. "
-        + "If not passed default schema will be used\n"
-        + "  -c <cellSize> : cell size to use for striped encoding files."
-        + " If not passed default cellsize of "
-        + HdfsConstants.BLOCK_STRIPED_CELL_SIZE + " will be used\n"
+        + "  -s <policyName> : erasure coding policy name to encode files. "
+        + "If not passed the default policy will be used\n"
         + "  <path>  : Path to an empty directory. Under this directory "
-        + "files will be encoded using specified schema";
-    private String schemaName;
-    private int cellSize = 0;
-    private ECSchema schema = null;
+        + "files will be encoded using specified erasure coding policy";
+    private String ecPolicyName;
+    private ErasureCodingPolicy ecPolicy = null;
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
-      schemaName = StringUtils.popOptionWithArgument("-s", args);
-      String cellSizeStr = StringUtils.popOptionWithArgument("-c", args);
-      if (cellSizeStr != null) {
-        cellSize = (int) StringUtils.TraditionalBinaryPrefix
-            .string2long(cellSizeStr);
-      }
+      ecPolicyName = StringUtils.popOptionWithArgument("-s", args);
       if (args.isEmpty()) {
         throw new HadoopIllegalArgumentException("<path> is missing");
       }
@@ -119,29 +107,29 @@ public abstract class ECCommand extends Command {
       super.processPath(item);
       DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
       try {
-        if (schemaName != null) {
-          ECSchema[] ecSchemas = dfs.getClient().getECSchemas();
-          for (ECSchema ecSchema : ecSchemas) {
-            if (schemaName.equals(ecSchema.getSchemaName())) {
-              schema = ecSchema;
+        if (ecPolicyName != null) {
+          ErasureCodingPolicy[] ecPolicies = 
dfs.getClient().getErasureCodingPolicies();
+          for (ErasureCodingPolicy ecPolicy : ecPolicies) {
+            if (ecPolicyName.equals(ecPolicy.getName())) {
+              this.ecPolicy = ecPolicy;
               break;
             }
           }
-          if (schema == null) {
+          if (ecPolicy == null) {
             StringBuilder sb = new StringBuilder();
-            sb.append("Schema '");
-            sb.append(schemaName);
-            sb.append("' does not match any of the supported schemas.");
+            sb.append("Policy '");
+            sb.append(ecPolicyName);
+            sb.append("' does not match any of the supported policies.");
             sb.append(" Please select any one of ");
-            List<String> schemaNames = new ArrayList<String>();
-            for (ECSchema ecSchema : ecSchemas) {
-              schemaNames.add(ecSchema.getSchemaName());
+            List<String> ecPolicyNames = new ArrayList<String>();
+            for (ErasureCodingPolicy ecPolicy : ecPolicies) {
+              ecPolicyNames.add(ecPolicy.getName());
             }
-            sb.append(schemaNames);
+            sb.append(ecPolicyNames);
             throw new HadoopIllegalArgumentException(sb.toString());
           }
         }
-        dfs.createErasureCodingZone(item.path, schema, cellSize);
+        dfs.createErasureCodingZone(item.path, ecPolicy);
         out.println("EC Zone created successfully at " + item.path);
       } catch (IOException e) {
         throw new IOException("Unable to create EC zone for the path "
@@ -188,13 +176,13 @@ public abstract class ECCommand extends Command {
   }
 
   /**
-   * List all supported EC Schemas
+   * List all supported erasure coding policies
    */
-  static class ListECSchemas extends ECCommand {
-    public static final String NAME = "listSchemas";
+  static class ListPolicies extends ECCommand {
+    public static final String NAME = "listPolicies";
     public static final String USAGE = "";
     public static final String DESCRIPTION = 
-        "Get the list of ECSchemas supported\n";
+        "Get the list of erasure coding policies supported\n";
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
@@ -209,14 +197,14 @@ public abstract class ECCommand extends Command {
       }
       DistributedFileSystem dfs = (DistributedFileSystem) fs;
 
-      ECSchema[] ecSchemas = dfs.getClient().getECSchemas();
+      ErasureCodingPolicy[] ecPolicies = 
dfs.getClient().getErasureCodingPolicies();
       StringBuilder sb = new StringBuilder();
       int i = 0;
-      while (i < ecSchemas.length) {
-        ECSchema ecSchema = ecSchemas[i];
-        sb.append(ecSchema.getSchemaName());
+      while (i < ecPolicies.length) {
+        ErasureCodingPolicy ecPolicy = ecPolicies[i];
+        sb.append(ecPolicy.getName());
         i++;
-        if (i < ecSchemas.length) {
+        if (i < ecPolicies.length) {
           sb.append(", ");
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
index 4dc94a0..4fded73 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 import org.apache.hadoop.security.token.Token;
 
@@ -318,7 +318,7 @@ public class StripedBlockUtil {
   }
 
   /**
-   * Decode based on the given input buffers and schema.
+   * Decode based on the given input buffers and erasure coding policy.
    */
   public static void decodeAndFillBuffer(final byte[][] decodeInputs,
       AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum,
@@ -355,20 +355,20 @@ public class StripedBlockUtil {
    * by stateful read and uses ByteBuffer as reading target buffer. Besides the
    * read range is within a single stripe thus the calculation logic is 
simpler.
    */
-  public static AlignedStripe[] divideOneStripe(ECSchema ecSchema,
+  public static AlignedStripe[] divideOneStripe(ErasureCodingPolicy ecPolicy,
       int cellSize, LocatedStripedBlock blockGroup, long 
rangeStartInBlockGroup,
       long rangeEndInBlockGroup, ByteBuffer buf) {
-    final int dataBlkNum = ecSchema.getNumDataUnits();
+    final int dataBlkNum = ecPolicy.getNumDataUnits();
     // Step 1: map the byte range to StripingCells
-    StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, cellSize,
+    StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize,
         blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup);
 
     // Step 2: get the unmerged ranges on each internal block
-    VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cellSize,
+    VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize,
         cells);
 
     // Step 3: merge into stripes
-    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges);
+    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges);
 
     // Step 4: calculate each chunk's position in destination buffer. Since the
     // whole read range is within a single stripe, the logic is simpler here.
@@ -400,7 +400,7 @@ public class StripedBlockUtil {
   /**
    * This method divides a requested byte range into an array of inclusive
    * {@link AlignedStripe}.
-   * @param ecSchema The codec schema for the file, which carries the numbers
+   * @param ecPolicy The codec policy for the file, which carries the numbers
    *                 of data / parity blocks
    * @param cellSize Cell size of stripe
    * @param blockGroup The striped block group
@@ -412,24 +412,24 @@ public class StripedBlockUtil {
    * At most 5 stripes will be generated from each logical range, as
    * demonstrated in the header of {@link AlignedStripe}.
    */
-  public static AlignedStripe[] divideByteRangeIntoStripes(ECSchema ecSchema,
+  public static AlignedStripe[] divideByteRangeIntoStripes(ErasureCodingPolicy 
ecPolicy,
       int cellSize, LocatedStripedBlock blockGroup,
       long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf,
       int offsetInBuf) {
 
     // Step 0: analyze range and calculate basic parameters
-    final int dataBlkNum = ecSchema.getNumDataUnits();
+    final int dataBlkNum = ecPolicy.getNumDataUnits();
 
     // Step 1: map the byte range to StripingCells
-    StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, cellSize,
+    StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize,
         blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup);
 
     // Step 2: get the unmerged ranges on each internal block
-    VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cellSize,
+    VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize,
         cells);
 
     // Step 3: merge into at most 5 stripes
-    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges);
+    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges);
 
     // Step 4: calculate each chunk's position in destination buffer
     calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf, offsetInBuf);
@@ -446,7 +446,7 @@ public class StripedBlockUtil {
    * used by {@link DFSStripedOutputStream} in encoding
    */
   @VisibleForTesting
-  private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema,
+  private static StripingCell[] 
getStripingCellsOfByteRange(ErasureCodingPolicy ecPolicy,
       int cellSize, LocatedStripedBlock blockGroup,
       long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
     Preconditions.checkArgument(
@@ -461,16 +461,16 @@ public class StripedBlockUtil {
     final int firstCellOffset = (int) (rangeStartInBlockGroup % cellSize);
     final int firstCellSize =
         (int) Math.min(cellSize - (rangeStartInBlockGroup % cellSize), len);
-    cells[0] = new StripingCell(ecSchema, firstCellSize, firstCellIdxInBG,
+    cells[0] = new StripingCell(ecPolicy, firstCellSize, firstCellIdxInBG,
         firstCellOffset);
     if (lastCellIdxInBG != firstCellIdxInBG) {
       final int lastCellSize = (int) (rangeEndInBlockGroup % cellSize) + 1;
-      cells[numCells - 1] = new StripingCell(ecSchema, lastCellSize,
+      cells[numCells - 1] = new StripingCell(ecPolicy, lastCellSize,
           lastCellIdxInBG, 0);
     }
 
     for (int i = 1; i < numCells - 1; i++) {
-      cells[i] = new StripingCell(ecSchema, cellSize, i + firstCellIdxInBG, 0);
+      cells[i] = new StripingCell(ecPolicy, cellSize, i + firstCellIdxInBG, 0);
     }
 
     return cells;
@@ -481,10 +481,10 @@ public class StripedBlockUtil {
    * the physical byte range (inclusive) on each stored internal block.
    */
   @VisibleForTesting
-  private static VerticalRange[] getRangesForInternalBlocks(ECSchema ecSchema,
+  private static VerticalRange[] 
getRangesForInternalBlocks(ErasureCodingPolicy ecPolicy,
       int cellSize, StripingCell[] cells) {
-    int dataBlkNum = ecSchema.getNumDataUnits();
-    int parityBlkNum = ecSchema.getNumParityUnits();
+    int dataBlkNum = ecPolicy.getNumDataUnits();
+    int parityBlkNum = ecPolicy.getNumParityUnits();
 
     VerticalRange ranges[] = new VerticalRange[dataBlkNum + parityBlkNum];
 
@@ -521,9 +521,9 @@ public class StripedBlockUtil {
    * {@link AlignedStripe} instances.
    */
   private static AlignedStripe[] mergeRangesForInternalBlocks(
-      ECSchema ecSchema, VerticalRange[] ranges) {
-    int dataBlkNum = ecSchema.getNumDataUnits();
-    int parityBlkNum = ecSchema.getNumParityUnits();
+      ErasureCodingPolicy ecPolicy, VerticalRange[] ranges) {
+    int dataBlkNum = ecPolicy.getNumDataUnits();
+    int parityBlkNum = ecPolicy.getNumParityUnits();
     List<AlignedStripe> stripes = new ArrayList<>();
     SortedSet<Long> stripePoints = new TreeSet<>();
     for (VerticalRange r : ranges) {
@@ -628,7 +628,7 @@ public class StripedBlockUtil {
    */
   @VisibleForTesting
   static class StripingCell {
-    final ECSchema schema;
+    final ErasureCodingPolicy ecPolicy;
     /** Logical order in a block group, used when doing I/O to a block group */
     final int idxInBlkGroup;
     final int idxInInternalBlk;
@@ -642,13 +642,13 @@ public class StripedBlockUtil {
     final int offset;
     final int size;
 
-    StripingCell(ECSchema ecSchema, int cellSize, int idxInBlkGroup,
+    StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, int idxInBlkGroup,
         int offset) {
-      this.schema = ecSchema;
+      this.ecPolicy = ecPolicy;
       this.idxInBlkGroup = idxInBlkGroup;
-      this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits();
+      this.idxInInternalBlk = idxInBlkGroup / ecPolicy.getNumDataUnits();
       this.idxInStripe = idxInBlkGroup -
-          this.idxInInternalBlk * ecSchema.getNumDataUnits();
+          this.idxInInternalBlk * ecPolicy.getNumDataUnits();
       this.offset = offset;
       this.size = cellSize;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/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 2b64ca0..a998147 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -863,8 +863,8 @@ service ClientNamenodeProtocol {
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
       returns(GetEditsFromTxidResponseProto);
-  rpc getECSchemas(GetECSchemasRequestProto)
-      returns(GetECSchemasResponseProto);
+  rpc getErasureCodingPolicies(GetErasureCodingPoliciesRequestProto)
+      returns(GetErasureCodingPoliciesResponseProto);
   rpc getErasureCodingZone(GetErasureCodingZoneRequestProto)
       returns(GetErasureCodingZoneResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/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 56bb7a2..d27f782 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -28,24 +28,22 @@ import "hdfs.proto";
  */
 message ErasureCodingZoneProto {
   required string dir = 1;
-  required ECSchemaProto schema = 2;
-  required uint32 cellSize = 3;
+  required ErasureCodingPolicyProto ecPolicy = 2;
 }
 
 message CreateErasureCodingZoneRequestProto {
   required string src = 1;
-  optional ECSchemaProto schema = 2;
-  optional uint32 cellSize = 3;
+  optional ErasureCodingPolicyProto ecPolicy = 2;
 }
 
 message CreateErasureCodingZoneResponseProto {
 }
 
-message GetECSchemasRequestProto { // void request
+message GetErasureCodingPoliciesRequestProto { // void request
 }
 
-message GetECSchemasResponseProto {
-  repeated ECSchemaProto schemas = 1;
+message GetErasureCodingPoliciesResponseProto {
+  repeated ErasureCodingPolicyProto ecPolicies = 1;
 }
 
 message GetErasureCodingZoneRequestProto {
@@ -66,6 +64,5 @@ message BlockECRecoveryInfoProto {
   required StorageUuidsProto targetStorageUuids = 4;
   required StorageTypesProto targetStorageTypes = 5;
   repeated uint32 liveBlockIndices = 6;
-  required ECSchemaProto ecSchema = 7;
-  required uint32 cellSize = 8;
-}
\ No newline at end of file
+  required ErasureCodingPolicyProto ecPolicy = 7;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index 3233f66..43a1fc5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -141,7 +141,6 @@ message INodeSection {
     optional XAttrFeatureProto xAttrs = 9;
     optional uint32 storagePolicyID = 10;
     optional bool isStriped = 11;
-    optional uint64 stripingCellSize = 12;
   }
 
   message QuotaByStorageTypeEntryProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index d2cb665..63fe90c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -306,8 +306,7 @@ message LocatedBlocksProto {
   optional FileEncryptionInfoProto fileEncryptionInfo = 6;
 
   // Optional field for erasure coding
-  optional ECSchemaProto eCSchema = 7;
-  optional uint32 stripeCellSize = 8;
+  optional ErasureCodingPolicyProto ecPolicy = 7;
 }
 
 /**
@@ -322,11 +321,16 @@ message ECSchemaOptionEntryProto {
  * ECSchema for erasurecoding
  */
 message ECSchemaProto {
-  required string schemaName = 1;
-  required string codecName = 2;
-  required uint32 dataUnits = 3;
-  required uint32 parityUnits = 4;
-  repeated ECSchemaOptionEntryProto options = 5;
+  required string codecName = 1;
+  required uint32 dataUnits = 2;
+  required uint32 parityUnits = 3;
+  repeated ECSchemaOptionEntryProto options = 4;
+}
+
+message ErasureCodingPolicyProto {
+  required string name = 1;
+  required ECSchemaProto schema = 2;
+  required uint32 cellSize = 3;
 }
 
 /**
@@ -365,8 +369,7 @@ message HdfsFileStatusProto {
   optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
 
   // Optional field for erasure coding
-  optional ECSchemaProto ecSchema = 17;
-  optional uint32 stripeCellSize = 18;
+  optional ErasureCodingPolicyProto ecPolicy = 17;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
index 5f01ea2..dfefb66 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
@@ -1,3 +1,4 @@
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 0258591..28283f3 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -1875,7 +1875,7 @@ public class DFSTestUtil {
       assert dir != null;
       dfs.mkdirs(dir);
       try {
-        dfs.getClient().createErasureCodingZone(dir.toString(), null, 0);
+        dfs.getClient().createErasureCodingZone(dir.toString(), null);
       } catch (IOException e) {
         if (!e.getMessage().contains("non-empty directory")) {
           throw e;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index 55a335f..df2643a 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -255,12 +255,12 @@ public class TestDFSClientRetries {
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null, (byte) 0, null, 
0)).when(mockNN).getFileInfo(anyString());
+                1010, 0, null, (byte) 0, 
null)).when(mockNN).getFileInfo(anyString());
     
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null, (byte) 0, null, 0))
+                1010, 0, null, (byte) 0, null))
         .when(mockNN)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),
@@ -496,7 +496,7 @@ public class TestDFSClientRetries {
       badBlocks.add(badLocatedBlock);
       return new LocatedBlocks(goodBlockList.getFileLength(), false,
                                badBlocks, null, true,
-                               null, null, 0);
+                               null, null);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/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 baf6106..ed3c110 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
@@ -35,10 +35,10 @@ import static org.junit.Assert.assertTrue;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
-import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 import org.junit.After;
 import org.junit.Assert;
@@ -59,7 +59,7 @@ public class TestDFSStripedInputStream {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
-  private final ECSchema schema = 
ErasureCodingSchemaManager.getSystemDefaultSchema();
+  private final ErasureCodingPolicy ecPolicy = 
ErasureCodingPolicyManager.getSystemDefaultPolicy();
   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;
@@ -79,7 +79,7 @@ public class TestDFSStripedInputStream {
     }
     fs = cluster.getFileSystem();
     fs.mkdirs(dirPath);
-    fs.getClient().createErasureCodingZone(dirPath.toString(), null, CELLSIZE);
+    fs.getClient().createErasureCodingZone(dirPath.toString(), null);
   }
 
   @After
@@ -100,7 +100,7 @@ public class TestDFSStripedInputStream {
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
     final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
-        filePath.toString(), false, schema, CELLSIZE, null);
+        filePath.toString(), false, ecPolicy, null);
 
     List<LocatedBlock> lbList = lbs.getLocatedBlocks();
     for (LocatedBlock aLbList : lbList) {
@@ -152,7 +152,7 @@ public class TestDFSStripedInputStream {
       }
     }
     DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
-        filePath.toString(), false, schema, CELLSIZE, null);
+        filePath.toString(), false, ecPolicy, null);
 
     int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102,
         CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102,
@@ -194,7 +194,7 @@ public class TestDFSStripedInputStream {
     }
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(), false,
-            ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE, 
null);
+            ErasureCodingPolicyManager.getSystemDefaultPolicy(), null);
     int readSize = BLOCK_GROUP_SIZE;
     byte[] readBuffer = new byte[readSize];
     byte[] expected = new byte[readSize];
@@ -292,7 +292,7 @@ public class TestDFSStripedInputStream {
 
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(),
-            false, schema, CELLSIZE, null);
+            false, ecPolicy, null);
 
     byte[] expected = new byte[fileSize];
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index 3f40dee..5cab978 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -68,7 +68,7 @@ public class TestDFSStripedOutputStream {
     conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null, 0);
+    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index 6594ae1..fed9f16 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -79,7 +79,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     cluster.waitActive();
     dfs = cluster.getFileSystem();
     dfs.mkdirs(dir);
-    dfs.createErasureCodingZone(dir, null, 0);
+    dfs.createErasureCodingZone(dir, null);
   }
 
   private void tearDown() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index 4233a1c..6a24685 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -110,7 +110,7 @@ public class TestDFSUtil {
     l2.setCorrupt(true);
 
     List<LocatedBlock> ls = Arrays.asList(l1, l2);
-    LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null, null, 
0);
+    LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null, null);
 
     BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
index 9cdb763..989e9fc 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
@@ -81,7 +81,7 @@ public class TestDatanodeConfig {
   public void testDataDirectories() throws IOException {
     File dataDir = new File(BASE_DIR, "data").getCanonicalFile();
     Configuration conf = cluster.getConfiguration(0);
-    // 1. Test unsupported schema. Only "file:" is supported.
+    // 1. Test unsupported ecPolicy. Only "file:" is supported.
     String dnDir = makeURI("shv", null, fileAsURI(dataDir).getPath());
     conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dnDir);
     DataNode dn = null;
@@ -97,7 +97,7 @@ public class TestDatanodeConfig {
     }
     assertNull("Data-node startup should have failed.", dn);
 
-    // 2. Test "file:" schema and no schema (path-only). Both should work.
+    // 2. Test "file:" ecPolicy and no ecPolicy (path-only). Both should work.
     String dnDir1 = fileAsURI(dataDir).toString() + "1";
     String dnDir2 = makeURI("file", "localhost",
                     fileAsURI(dataDir).getPath() + "2");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d37a881/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
deleted file mode 100644
index 88198c9..0000000
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
+++ /dev/null
@@ -1,54 +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;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestECSchemas {
-  private MiniDFSCluster cluster;
-
-  @Before
-  public void before() throws IOException {
-    cluster = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(0)
-        .build();
-    cluster.waitActive();
-  }
-
-  @After
-  public void after() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testGetECSchemas() throws Exception {
-    ECSchema[] ecSchemas = cluster.getFileSystem().getClient().getECSchemas();
-    assertNotNull(ecSchemas);
-    assertTrue("Should have at least one schema", ecSchemas.length > 0);
-  }
-}

Reply via email to