Merge master to EC branch HDDS-3816-ec
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/ECXceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/ECXceiverClientGrpc.java
new file mode 100644
index 0000000..4873fdd
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/ECXceiverClientGrpc.java
@@ -0,0 +1,60 @@
+/*
+ * 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.hdds.scm;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+
+import java.security.cert.X509Certificate;
+import java.util.List;
+
+/**
+ * {@link XceiverClientSpi} implementation to work specifically with EC
+ * related requests. The only difference at the moment from the basic
+ * {@link XceiverClientGrpc} is that this implementation does async calls when
+ * a write request is posted via the sendCommandAsync method.
+ *
+ * @see https://issues.apache.org/jira/browse/HDDS-5954
+ */
+public class ECXceiverClientGrpc extends XceiverClientGrpc {
+
+  public ECXceiverClientGrpc(
+      Pipeline pipeline,
+      ConfigurationSource config,
+      List<X509Certificate> caCerts) {
+    super(pipeline, config, caCerts);
+  }
+
+  /**
+   * For EC writes, due to outside syncronization points during writes, it is
+   * not necessary to block any async requests that are
+   * arriving via the
+   * {@link #sendCommandAsync(ContainerProtos.ContainerCommandRequestProto)}
+   * method.
+   *
+   * @param request the request we need the decision about
+   * @return false always to do not block async requests.
+   */
+  @Override
+  protected boolean shouldBlockAndWaitAsyncReply(
+      ContainerProtos.ContainerCommandRequestProto request) {
+    return false;
+  }
+}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java
index 064ce6e..63dd511 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java
@@ -136,6 +136,32 @@
       tags = ConfigTag.CLIENT)
   private boolean checksumVerify = true;
 
+  @Config(key = "max.ec.stripe.write.retries",
+      defaultValue = "10",
+      description = "Ozone EC client to retry stripe to new block group on" +
+          " failures.",
+      tags = ConfigTag.CLIENT)
+  private int maxECStripeWriteRetries = 10;
+
+  @Config(key = "exclude.nodes.expiry.time",
+      defaultValue = "600000",
+      description = "Time after which an excluded node is reconsidered for" +
+          " writes in EC. If the value is zero, the node is excluded for the" +
+          " life of the client",
+      tags = ConfigTag.CLIENT)
+  private long excludeNodesExpiryTime = 10 * 60 * 1000;
+
+  @Config(key = "ec.reconstruct.stripe.read.pool.limit",
+      defaultValue = "30",
+      description = "Thread pool max size for parallelly read" +
+          " available ec chunks to reconstruct the whole stripe.",
+      tags = ConfigTag.CLIENT)
+  // For the largest recommended EC policy rs-10-4-1024k,
+  // 10 chunks are required at least for stripe reconstruction,
+  // so 1 core thread for each chunk and
+  // 3 concurrent stripe read should be enough.
+  private int ecReconstructStripeReadPoolLimit = 10 * 3;
+
   @Config(key = "checksum.combine.mode",
       defaultValue = "COMPOSITE_CRC",
       description = "The combined checksum type [MD5MD5CRC / COMPOSITE_CRC] "
@@ -250,6 +276,14 @@
     this.checksumVerify = checksumVerify;
   }
 
+  public int getMaxECStripeWriteRetries() {
+    return this.maxECStripeWriteRetries;
+  }
+
+  public long getExcludeNodesExpiryTime() {
+    return excludeNodesExpiryTime;
+  }
+
   public int getBufferIncrement() {
     return bufferIncrement;
   }
@@ -265,4 +299,12 @@
           ChecksumCombineMode.COMPOSITE_CRC.name());
     }
   }
+
+  public void setEcReconstructStripeReadPoolLimit(int poolLimit) {
+    this.ecReconstructStripeReadPoolLimit = poolLimit;
+  }
+
+  public int getEcReconstructStripeReadPoolLimit() {
+    return ecReconstructStripeReadPoolLimit;
+  }
 }
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
index a268495..eea2bf3 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
@@ -460,7 +460,7 @@
       // served out of order over XceiverClientGrpc. This needs to be fixed
       // if this API is to be used for I/O path. Currently, this is not
       // used for Read/Write Operation but for tests.
-      if (!HddsUtils.isReadOnly(request)) {
+      if (shouldBlockAndWaitAsyncReply(request)) {
         asyncReply.getResponse().get();
       }
       return asyncReply;
@@ -470,6 +470,21 @@
     }
   }
 
+  /**
+   * During data writes the ordering of WriteChunk and PutBlock is not ensured
+   * by any outside logic, therefore in this original implementation, all reads
+   * and writes are synchronized.
+   * This method is providing the possibility for subclasses to override this
+   * behaviour.
+   *
+   * @param request the request we need the decision about
+   * @return true if the request is a write request.
+   */
+  protected boolean shouldBlockAndWaitAsyncReply(
+      ContainerCommandRequestProto request) {
+    return !HddsUtils.isReadOnly(request);
+  }
+
   @VisibleForTesting
   public XceiverClientReply sendCommandAsync(
       ContainerCommandRequestProto request, DatanodeDetails dn)
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
index f016972..ee4edec 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
@@ -29,6 +29,7 @@
 import org.apache.hadoop.hdds.conf.ConfigGroup;
 import org.apache.hadoop.hdds.conf.ConfigType;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -230,6 +231,9 @@
             case STAND_ALONE:
               client = new XceiverClientGrpc(pipeline, conf, caCerts);
               break;
+            case EC:
+              client = new ECXceiverClientGrpc(pipeline, conf, caCerts);
+              break;
             case CHAINED:
             default:
               throw new IOException("not implemented" + pipeline.getType());
@@ -246,9 +250,16 @@
 
   private String getPipelineCacheKey(Pipeline pipeline, boolean forRead) {
     String key = pipeline.getId().getId().toString() + pipeline.getType();
-    if (topologyAwareRead && forRead) {
+    boolean isEC = pipeline.getReplicationConfig()
+        .getReplicationType() == HddsProtos.ReplicationType.EC;
+    if (topologyAwareRead && forRead || isEC) {
       try {
         key += pipeline.getClosestNode().getHostName();
+        if (isEC) {
+          // Currently EC uses standalone client.
+          key += pipeline.getClosestNode()
+              .getPort(DatanodeDetails.Port.Name.STANDALONE);
+        }
       } catch (IOException e) {
         LOG.error("Failed to get closest node to create pipeline cache key:" +
             e.getMessage());
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java
new file mode 100644
index 0000000..5be2b07
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.storage;
+
+import org.apache.hadoop.hdds.client.BlockID;
+
+/**
+ * Abstract class used as an interface for input streams related to Ozone
+ * blocks.
+ */
+public abstract class BlockExtendedInputStream  extends ExtendedInputStream {
+
+  public abstract BlockID getBlockID();
+
+  public abstract long getRemaining();
+
+  public abstract long getLength();
+
+}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
index a6f22d4..07a444a 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
@@ -21,7 +21,6 @@
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -29,9 +28,6 @@
 import java.util.function.Function;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.ByteBufferReadable;
-import org.apache.hadoop.fs.CanUnbuffer;
-import org.apache.hadoop.fs.Seekable;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
@@ -59,14 +55,11 @@
  * This class encapsulates all state management for iterating
  * through the sequence of chunks through {@link ChunkInputStream}.
  */
-public class BlockInputStream extends InputStream
-    implements Seekable, CanUnbuffer, ByteBufferReadable {
+public class BlockInputStream extends BlockExtendedInputStream {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(BlockInputStream.class);
 
-  private static final int EOF = -1;
-
   private final BlockID blockID;
   private final long length;
   private Pipeline pipeline;
@@ -219,7 +212,8 @@
   protected List<ChunkInfo> getChunkInfos() throws IOException {
     // irrespective of the container state, we will always read via Standalone
     // protocol.
-    if (pipeline.getType() != HddsProtos.ReplicationType.STAND_ALONE) {
+    if (pipeline.getType() != HddsProtos.ReplicationType.STAND_ALONE && pipeline
+        .getType() != HddsProtos.ReplicationType.EC) {
       pipeline = Pipeline.newBuilder(pipeline)
           .setReplicationConfig(StandaloneReplicationConfig.getInstance(
               ReplicationConfig
@@ -269,46 +263,14 @@
         xceiverClientFactory, () -> pipeline, verifyChecksum, token);
   }
 
+  @Override
   public synchronized long getRemaining() {
     return length - getPos();
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
-  public synchronized int read() throws IOException {
-    byte[] buf = new byte[1];
-    if (read(buf, 0, 1) == EOF) {
-      return EOF;
-    }
-    return Byte.toUnsignedInt(buf[0]);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public synchronized int read(byte[] b, int off, int len) throws IOException {
-    ByteReaderStrategy strategy = new ByteArrayReader(b, off, len);
-    if (len == 0) {
-      return 0;
-    }
-    return readWithStrategy(strategy);
-  }
-
-  @Override
-  public synchronized int read(ByteBuffer byteBuffer) throws IOException {
-    ByteReaderStrategy strategy = new ByteBufferReader(byteBuffer);
-    int len = strategy.getTargetLength();
-    if (len == 0) {
-      return 0;
-    }
-    return readWithStrategy(strategy);
-  }
-
-  synchronized int readWithStrategy(ByteReaderStrategy strategy) throws
-      IOException {
+  protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+      throws IOException {
     Preconditions.checkArgument(strategy != null);
     if (!initialized) {
       initialize();
@@ -478,10 +440,6 @@
     }
   }
 
-  public synchronized void resetPosition() {
-    this.blockPosition = 0;
-  }
-
   /**
    * Checks if the stream is open.  If not, throw an exception.
    *
@@ -493,10 +451,12 @@
     }
   }
 
+  @Override
   public BlockID getBlockID() {
     return blockID;
   }
 
+  @Override
   public long getLength() {
     return length;
   }
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
index 8b3f817..b86464c 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
@@ -119,6 +119,7 @@
   //current buffer allocated to write
   private ChunkBuffer currentBuffer;
   private final Token<? extends TokenIdentifier> token;
+  private int replicationIndex;
 
   /**
    * Creates a new BlockOutputStream.
@@ -148,6 +149,8 @@
     this.bufferPool = bufferPool;
     this.token = token;
 
+    replicationIndex = pipeline.getReplicaIndex(pipeline.getClosestNode());
+
     //number of buffers used before doing a flush
     refreshCurrentBuffer();
     flushPeriod = (int) (config.getStreamBufferFlushSize() / config
@@ -210,6 +213,22 @@
     return ioException.get();
   }
 
+  XceiverClientSpi getXceiverClientSpi() {
+    return this.xceiverClient;
+  }
+
+  BlockData.Builder getContainerBlockData() {
+    return this.containerBlockData;
+  }
+
+  Token<? extends TokenIdentifier> getToken() {
+    return this.token;
+  }
+
+  ExecutorService getResponseExecutor() {
+    return this.responseExecutor;
+  }
+
   @Override
   public void write(int b) throws IOException {
     checkOpen();
@@ -396,7 +415,7 @@
    * @param force true if no data was written since most recent putBlock and
    *            stream is being closed
    */
-  private CompletableFuture<ContainerProtos.
+  CompletableFuture<ContainerProtos.
       ContainerCommandResponseProto> executePutBlock(boolean close,
       boolean force) throws IOException {
     checkOpen();
@@ -568,7 +587,7 @@
   void waitOnFlushFutures() throws InterruptedException, ExecutionException {
   }
 
-  private void validateResponse(
+  void validateResponse(
       ContainerProtos.ContainerCommandResponseProto responseProto)
       throws IOException {
     try {
@@ -587,7 +606,7 @@
   }
 
 
-  private void setIoException(Exception e) {
+  public void setIoException(Exception e) {
     IOException ioe = getIoException();
     if (ioe == null) {
       IOException exception =  new IOException(EXCEPTION_MSG + e.toString(), e);
@@ -623,7 +642,7 @@
    *
    * @throws IOException if stream is closed
    */
-  private void checkOpen() throws IOException {
+  void checkOpen() throws IOException {
     if (isClosed()) {
       throw new IOException("BlockOutputStream has been closed.");
     } else if (getIoException() != null) {
@@ -643,8 +662,10 @@
    * @throws IOException if there is an I/O error while performing the call
    * @throws OzoneChecksumException if there is an error while computing
    * checksum
+   * @return
    */
-  private void writeChunkToContainer(ChunkBuffer chunk) throws IOException {
+  CompletableFuture<ContainerCommandResponseProto> writeChunkToContainer(
+      ChunkBuffer chunk) throws IOException {
     int effectiveChunkSize = chunk.remaining();
     final long offset = chunkOffset.getAndAdd(effectiveChunkSize);
     final ByteString data = chunk.toByteString(
@@ -664,31 +685,34 @@
 
     try {
       XceiverClientReply asyncReply = writeChunkAsync(xceiverClient, chunkInfo,
-          blockID.get(), data, token);
-      CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
-          asyncReply.getResponse();
-      future.thenApplyAsync(e -> {
-        try {
-          validateResponse(e);
-        } catch (IOException sce) {
-          future.completeExceptionally(sce);
-        }
-        return e;
-      }, responseExecutor).exceptionally(e -> {
-        String msg = "Failed to write chunk " + chunkInfo.getChunkName() + " " +
-            "into block " + blockID;
-        LOG.debug("{}, exception: {}", msg, e.getLocalizedMessage());
-        CompletionException ce = new CompletionException(msg, e);
-        setIoException(ce);
-        throw ce;
-      });
+          blockID.get(), data, token, replicationIndex);
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+          respFuture = asyncReply.getResponse();
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+          validateFuture = respFuture.thenApplyAsync(e -> {
+            try {
+              validateResponse(e);
+            } catch (IOException sce) {
+              respFuture.completeExceptionally(sce);
+            }
+            return e;
+          }, responseExecutor).exceptionally(e -> {
+            String msg = "Failed to write chunk " + chunkInfo.getChunkName() +
+                " into block " + blockID;
+            LOG.debug("{}, exception: {}", msg, e.getLocalizedMessage());
+            CompletionException ce = new CompletionException(msg, e);
+            setIoException(ce);
+            throw ce;
+          });
+      containerBlockData.addChunks(chunkInfo);
+      return validateFuture;
     } catch (IOException | ExecutionException e) {
       throw new IOException(EXCEPTION_MSG + e.toString(), e);
     } catch (InterruptedException ex) {
       Thread.currentThread().interrupt();
       handleInterruptedException(ex, false);
     }
-    containerBlockData.addChunks(chunkInfo);
+    return null;
   }
 
   @VisibleForTesting
@@ -704,7 +728,7 @@
    * handle ExecutionException else skip it.
    * @throws IOException
    */
-  private void handleInterruptedException(Exception ex,
+  void handleInterruptedException(Exception ex,
       boolean processExecutionException)
       throws IOException {
     LOG.error("Command execution was interrupted.");
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java
new file mode 100644
index 0000000..b2f86ac
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java
@@ -0,0 +1,177 @@
+/*
+ * 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.hdds.scm.storage;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * Handles the chunk EC writes for an EC internal block.
+ */
+public class ECBlockOutputStream extends BlockOutputStream {
+
+  private final DatanodeDetails datanodeDetails;
+  private CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+      currentChunkRspFuture = null;
+
+  private CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+      putBlkRspFuture = null;
+  /**
+   * Creates a new ECBlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public ECBlockOutputStream(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    super(blockID, xceiverClientManager,
+        pipeline, bufferPool, config, token);
+    // In EC stream, there will be only one node in pipeline.
+    this.datanodeDetails = pipeline.getClosestNode();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    this.currentChunkRspFuture =
+        writeChunkToContainer(ChunkBuffer.wrap(ByteBuffer.wrap(b, off, len)));
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  public CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+
+    CompletableFuture<ContainerProtos.
+        ContainerCommandResponseProto> flushFuture = null;
+    try {
+      ContainerProtos.BlockData blockData = getContainerBlockData().build();
+      XceiverClientReply asyncReply =
+          putBlockAsync(getXceiverClient(), blockData, close, getToken());
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
+          asyncReply.getResponse();
+      flushFuture = future.thenApplyAsync(e -> {
+        try {
+          validateResponse(e);
+        } catch (IOException sce) {
+          throw new CompletionException(sce);
+        }
+        // if the ioException is not set, putBlock is successful
+        if (getIoException() == null) {
+          BlockID responseBlockID = BlockID.getFromProtobuf(
+              e.getPutBlock().getCommittedBlockLength().getBlockID());
+          Preconditions.checkState(getBlockID().getContainerBlockID()
+              .equals(responseBlockID.getContainerBlockID()));
+        }
+        return e;
+      }, getResponseExecutor()).exceptionally(e -> {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("putBlock failed for blockID {} with exception {}",
+              getBlockID(), e.getLocalizedMessage());
+        }
+        CompletionException ce =  new CompletionException(e);
+        setIoException(ce);
+        throw ce;
+      });
+    } catch (IOException | ExecutionException e) {
+      throw new IOException(EXCEPTION_MSG + e.toString(), e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, false);
+    }
+    this.putBlkRspFuture = flushFuture;
+    return flushFuture;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    cleanup(false);
+  }
+
+  /**
+   * @return The current chunk writer response future.
+   */
+  public CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+      getCurrentChunkResponseFuture() {
+    return this.currentChunkRspFuture;
+  }
+
+  /**
+   * @return The current chunk putBlock response future.
+   */
+  public CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+      getCurrentPutBlkResponseFuture() {
+    return this.putBlkRspFuture;
+  }
+
+  /**
+   * Gets the target data node used in the current stream.
+   * @return DatanodeDetails
+   */
+  public DatanodeDetails getDatanodeDetails() {
+    return datanodeDetails;
+  }
+
+  @Override
+  void validateResponse(
+      ContainerProtos.ContainerCommandResponseProto responseProto)
+      throws IOException {
+    try {
+      // if the ioException is already set, it means a prev request has failed
+      // just throw the exception. The current operation will fail with the
+      // original error
+      IOException exception = getIoException();
+      if (exception != null) {
+        return;
+      }
+      ContainerProtocolCalls.validateContainerResponse(responseProto);
+    } catch (IOException sce) {
+      setIoException(sce);
+    }
+  }
+}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ExtendedInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ExtendedInputStream.java
new file mode 100644
index 0000000..de868c7
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ExtendedInputStream.java
@@ -0,0 +1,91 @@
+/*
+ * 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.hdds.scm.storage;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.fs.CanUnbuffer;
+import org.apache.hadoop.fs.Seekable;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+/**
+ * Abstact class which extends InputStream and some common interfaces used by
+ * various Ozone InputStream classes.
+ */
+public abstract class ExtendedInputStream extends InputStream
+    implements Seekable, CanUnbuffer, ByteBufferReadable {
+
+  protected static final int EOF = -1;
+
+  @Override
+  public synchronized int read() throws IOException {
+    byte[] buf = new byte[1];
+    if (read(buf, 0, 1) == EOF) {
+      return EOF;
+    }
+    return Byte.toUnsignedInt(buf[0]);
+  }
+
+  @Override
+  public synchronized int read(byte[] b, int off, int len) throws IOException {
+    ByteReaderStrategy strategy = new ByteArrayReader(b, off, len);
+    int bufferLen = strategy.getTargetLength();
+    if (bufferLen == 0) {
+      return 0;
+    }
+    return readWithStrategy(strategy);
+  }
+
+  @Override
+  public synchronized int read(ByteBuffer byteBuffer) throws IOException {
+    ByteReaderStrategy strategy = new ByteBufferReader(byteBuffer);
+    int bufferLen = strategy.getTargetLength();
+    if (bufferLen == 0) {
+      return 0;
+    }
+    return readWithStrategy(strategy);
+  }
+
+  /**
+   * This must be overridden by the extending classes to call read on the
+   * underlying stream they are reading from. The last stream in the chain (the
+   * one which provides the actual data) needs to provide a real read via the
+   * read methods. For example if a test is extending this class, then it will
+   * need to override both read methods above and provide a dummy
+   * readWithStrategy implementation, as it will never be called by the tests.
+   *
+   * @param strategy
+   * @return
+   * @throws IOException
+   */
+  protected abstract int readWithStrategy(ByteReaderStrategy strategy) throws
+      IOException;
+
+  @Override
+  public synchronized void seek(long l) throws IOException {
+    throw new NotImplementedException("Seek is not implemented");
+  }
+
+  @Override
+  public synchronized boolean seekToNewSource(long l) throws IOException {
+    return false;
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java
new file mode 100644
index 0000000..851cae9
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.client;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
+import java.util.Objects;
+
+/**
+ * Replication configuration for EC replication.
+ */
+public class DefaultReplicationConfig {
+
+  private ReplicationType type;
+  private ReplicationFactor factor;
+  private ECReplicationConfig ecReplicationConfig;
+
+  public DefaultReplicationConfig(ReplicationType type,
+      ReplicationFactor factor) {
+    this.type = type;
+    this.factor = factor;
+    this.ecReplicationConfig = null;
+  }
+
+  public DefaultReplicationConfig(ReplicationConfig replicationConfig) {
+    this.type =
+        ReplicationType.fromProto(replicationConfig.getReplicationType());
+    if (replicationConfig instanceof ECReplicationConfig) {
+      this.ecReplicationConfig = (ECReplicationConfig) replicationConfig;
+    } else {
+      this.factor =
+          ReplicationFactor.valueOf(replicationConfig.getRequiredNodes());
+    }
+  }
+
+  public DefaultReplicationConfig(ReplicationType type,
+      ECReplicationConfig ecReplicationConfig) {
+    this.type = type;
+    this.factor = null;
+    this.ecReplicationConfig = ecReplicationConfig;
+  }
+
+  public DefaultReplicationConfig(ReplicationType type,
+      ReplicationFactor factor, ECReplicationConfig ecReplicationConfig) {
+    this.type = type;
+    this.factor = factor;
+    this.ecReplicationConfig = ecReplicationConfig;
+  }
+
+  public DefaultReplicationConfig(
+      org.apache.hadoop.hdds.protocol.proto.HddsProtos.DefaultReplicationConfig
+          defaultReplicationConfig) {
+    this.type = ReplicationType.fromProto(defaultReplicationConfig.getType());
+    if (defaultReplicationConfig.hasEcReplicationConfig()) {
+      this.ecReplicationConfig = new ECReplicationConfig(
+          defaultReplicationConfig.getEcReplicationConfig());
+    } else {
+      this.factor =
+          ReplicationFactor.fromProto(defaultReplicationConfig.getFactor());
+    }
+  }
+
+  public ReplicationType getType() {
+    return this.type;
+  }
+
+  public ReplicationFactor getFactor() {
+    return this.factor;
+  }
+
+  public DefaultReplicationConfig copy() {
+    return new DefaultReplicationConfig(this.type, this.factor,
+        this.ecReplicationConfig);
+  }
+
+  public ECReplicationConfig getEcReplicationConfig() {
+    return this.ecReplicationConfig;
+  }
+
+  public int getRequiredNodes() {
+    if (this.type == ReplicationType.EC) {
+      return ecReplicationConfig.getRequiredNodes();
+    }
+    return this.factor.getValue();
+  }
+
+  public HddsProtos.DefaultReplicationConfig toProto() {
+    final HddsProtos.DefaultReplicationConfig.Builder builder =
+        HddsProtos.DefaultReplicationConfig.newBuilder()
+            .setType(ReplicationType.toProto(this.type));
+    if (this.factor != null) {
+      builder.setFactor(ReplicationFactor.toProto(this.factor));
+    }
+    if (this.ecReplicationConfig != null) {
+      builder.setEcReplicationConfig(this.ecReplicationConfig.toProto());
+    }
+    return builder.build();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    DefaultReplicationConfig that = (DefaultReplicationConfig) o;
+    return Objects.equals(type, that.type) && Objects
+        .equals(factor, that.factor) && Objects
+        .equals(ecReplicationConfig, ecReplicationConfig);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(type, factor, ecReplicationConfig);
+  }
+}
+
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java
new file mode 100644
index 0000000..deb17cd
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java
@@ -0,0 +1,207 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.client;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
+import java.util.EnumSet;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Replication configuration for EC replication.
+ */
+public class ECReplicationConfig implements ReplicationConfig {
+
+  public static final String EC_REPLICATION_PARAMS_DELIMITER = "-";
+
+  /**
+   * Enum defining the allowed list of ECCodecs.
+   */
+  public enum EcCodec {
+    RS, XOR;
+
+    @Override
+    public String toString() {
+      return name().toLowerCase();
+    }
+
+    public static String allValuesAsString() {
+      return EnumSet.allOf(EcCodec.class)
+          .stream()
+          .map(Enum::toString)
+          .collect(Collectors.joining(","));
+    }
+  }
+
+  // Acceptable patterns are like:
+  //   rs-3-2-1024k
+  //   RS-3-2-2048
+  //   XOR-10-4-4096K
+  private static final Pattern STRING_FORMAT
+      = Pattern.compile("([a-zA-Z]+)-(\\d+)-(\\d+)-(\\d+)((?:k|K))?");
+
+  private int data;
+
+  private int parity;
+
+  private int ecChunkSize = 1024 * 1024;
+
+  private EcCodec codec = EcCodec.RS;
+
+  public ECReplicationConfig(int data, int parity) {
+    this.data = data;
+    this.parity = parity;
+  }
+
+  public ECReplicationConfig(int data, int parity, EcCodec codec,
+      int ecChunkSize) {
+    this.data = data;
+    this.parity = parity;
+    this.codec = codec;
+    this.ecChunkSize = ecChunkSize;
+  }
+
+  /**
+   * Create an ECReplicationConfig object from a string representing the
+   * various parameters. Acceptable patterns are like:
+   *     rs-3-2-1024k
+   *     RS-3-2-2048
+   *     XOR-10-4-4096K
+   * IllegalArgumentException will be thrown if the passed string does not
+   * match the defined pattern.
+   * @param string
+   */
+  public ECReplicationConfig(String string) {
+    final Matcher matcher = STRING_FORMAT.matcher(string);
+    if (!matcher.matches()) {
+      throw new IllegalArgumentException("EC replication config should be " +
+          "defined in the form rs-3-2-1024k, rs-6-3-1024; or rs-10-4-1024k." +
+          " Provided configuration was: " + string);
+    }
+
+    try {
+      codec = EcCodec.valueOf(matcher.group(1).toUpperCase());
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("The codec " + matcher.group(1) +
+          " is invalid. It must be one of " + EcCodec.allValuesAsString() + ".",
+          e);
+    }
+
+    data = Integer.parseInt(matcher.group(2));
+    parity = Integer.parseInt(matcher.group(3));
+    if (data <= 0 || parity <= 0) {
+      throw new IllegalArgumentException("Data and parity part in EC " +
+          "replication config supposed to be positive numbers");
+    }
+
+    int chunkSize = Integer.parseInt((matcher.group(4)));
+    if (chunkSize <= 0) {
+      throw new IllegalArgumentException("The ecChunkSize (" + chunkSize +
+          ") be greater than zero");
+    }
+    if (matcher.group(5) != null) {
+      // The "k" modifier is present, so multiple by 1024
+      chunkSize = chunkSize * 1024;
+    }
+    ecChunkSize = chunkSize;
+  }
+
+  public ECReplicationConfig(
+      HddsProtos.ECReplicationConfig ecReplicationConfig) {
+    this.data = ecReplicationConfig.getData();
+    this.parity = ecReplicationConfig.getParity();
+    this.codec = EcCodec.valueOf(ecReplicationConfig.getCodec().toUpperCase());
+    this.ecChunkSize = ecReplicationConfig.getEcChunkSize();
+  }
+
+  @Override
+  public HddsProtos.ReplicationType getReplicationType() {
+    return HddsProtos.ReplicationType.EC;
+  }
+
+  @Override
+  public int getRequiredNodes() {
+    return data + parity;
+  }
+
+  @Override
+  @JsonIgnore
+  public String getReplication() {
+    return getCodec() + EC_REPLICATION_PARAMS_DELIMITER
+        + getData() + EC_REPLICATION_PARAMS_DELIMITER
+        + getParity() + EC_REPLICATION_PARAMS_DELIMITER
+        + getEcChunkSize();
+  }
+
+  public HddsProtos.ECReplicationConfig toProto() {
+    return HddsProtos.ECReplicationConfig.newBuilder()
+        .setData(data)
+        .setParity(parity)
+        .setCodec(codec.toString())
+        .setEcChunkSize(ecChunkSize)
+        .build();
+  }
+
+  public int getData() {
+    return data;
+  }
+
+  public int getParity() {
+    return parity;
+  }
+
+  public int getEcChunkSize() {
+    return ecChunkSize;
+  }
+
+  public EcCodec getCodec() {
+    return codec;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ECReplicationConfig that = (ECReplicationConfig) o;
+    return data == that.data && parity == that.parity
+        && codec == that.getCodec() && ecChunkSize == that.getEcChunkSize();
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(data, parity, codec, ecChunkSize);
+  }
+
+  @Override
+  public String toString() {
+    return HddsProtos.ReplicationType.EC + "/ECReplicationConfig{"
+        + "data=" + data
+        + ", parity=" + parity
+        + ", ecChunkSize=" + ecChunkSize
+        + ", codec=" + codec + '}';
+  }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java
index 6d294d1..b0c1dd2 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdds.client;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -92,6 +93,12 @@
   }
 
   @Override
+  @JsonIgnore
+  public String getReplication() {
+    return String.valueOf(replicationFactor);
+  }
+
+  @Override
   public boolean equals(Object o) {
     if (this == o) {
       return true;
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java
index bcf4ea6..c9a12a6 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java
@@ -76,6 +76,28 @@
     return parse(null, replication, config);
   }
 
+  /**
+   * Helper method to serialize from proto.
+   * <p>
+   * This uses either the old type/factor or the new ecConfig depends on the
+   * type.
+   */
+  static ReplicationConfig fromProto(
+      HddsProtos.ReplicationType type,
+      HddsProtos.ReplicationFactor factor,
+      HddsProtos.ECReplicationConfig ecConfig) {
+    switch (type) {
+    case EC:
+      return new ECReplicationConfig(ecConfig);
+    case RATIS:
+    case STAND_ALONE:
+      return fromProtoTypeAndFactor(type, factor);
+    default:
+      throw new UnsupportedOperationException(
+          "Not supported replication: " + type);
+    }
+  }
+
   static HddsProtos.ReplicationFactor getLegacyFactor(
       ReplicationConfig replicationConfig) {
     if (replicationConfig instanceof ReplicatedReplicationConfig) {
@@ -101,9 +123,12 @@
    */
   static ReplicationConfig adjustReplication(
       ReplicationConfig config, short replication, ConfigurationSource conf) {
-    return parse(
-        ReplicationType.valueOf(config.getReplicationType().toString()),
-        Short.toString(replication), conf);
+    ReplicationType replicationType =
+        ReplicationType.valueOf(config.getReplicationType().toString());
+    if (replicationType.equals(ReplicationType.EC)) {
+      return config;
+    }
+    return parse(replicationType, Short.toString(replication), conf);
   }
 
   /**
@@ -134,6 +159,21 @@
     replication = Objects.toString(replication,
         config.get(OZONE_REPLICATION, OZONE_REPLICATION_DEFAULT));
 
+    return parseWithoutFallback(type, replication, config);
+  }
+
+  static ReplicationConfig parseWithoutFallback(ReplicationType type,
+      String replication, ConfigurationSource config) {
+
+    if (replication == null) {
+      throw new IllegalArgumentException(
+          "Replication can't be null. Replication type passed was : " + type);
+    }
+    if (type == null) {
+      throw new IllegalArgumentException(
+          "Replication type must be specified for: " + replication);
+    }
+
     ReplicationConfig replicationConfig;
     switch (type) {
     case RATIS:
@@ -146,6 +186,8 @@
       }
       replicationConfig = fromTypeAndFactor(type, factor);
       break;
+    case EC:
+      return new ECReplicationConfig(replication);
     default:
       throw new RuntimeException("Replication type" + type + " can not"
           + "be parsed.");
@@ -168,4 +210,9 @@
    */
   int getRequiredNodes();
 
+  /**
+   * Returns the replication option in string format.
+   */
+  String getReplication();
+
 }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationFactor.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationFactor.java
index 8623a0e..a249040 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationFactor.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationFactor.java
@@ -71,6 +71,22 @@
     }
   }
 
+  public static HddsProtos.ReplicationFactor toProto(
+       ReplicationFactor replicationFactor) {
+    if (replicationFactor == null) {
+      return null;
+    }
+    switch (replicationFactor) {
+    case ONE:
+      return HddsProtos.ReplicationFactor.ONE;
+    case THREE:
+      return HddsProtos.ReplicationFactor.THREE;
+    default:
+      throw new IllegalArgumentException(
+          "Unsupported ProtoBuf replication factor: " + replicationFactor);
+    }
+  }
+
   /**
    * Returns integer representation of ReplicationFactor.
    * @return replication value
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java
index 11a8218..64969eac 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java
@@ -26,7 +26,8 @@
 public enum ReplicationType {
   RATIS,
   STAND_ALONE,
-  CHAINED;
+  CHAINED,
+  EC;
 
   public static ReplicationType fromProto(
       HddsProtos.ReplicationType replicationType) {
@@ -40,9 +41,31 @@
       return ReplicationType.STAND_ALONE;
     case CHAINED:
       return ReplicationType.CHAINED;
+    case EC:
+      return ReplicationType.EC;
     default:
       throw new IllegalArgumentException(
           "Unsupported ProtoBuf replication type: " + replicationType);
     }
   }
+
+  public static HddsProtos.ReplicationType toProto(
+       ReplicationType replicationType) {
+    if (replicationType == null) {
+      return null;
+    }
+    switch (replicationType) {
+    case RATIS:
+      return HddsProtos.ReplicationType.RATIS;
+    case STAND_ALONE:
+      return HddsProtos.ReplicationType.STAND_ALONE;
+    case CHAINED:
+      return HddsProtos.ReplicationType.CHAINED;
+    case EC:
+      return HddsProtos.ReplicationType.EC;
+    default:
+      throw new IllegalArgumentException(
+          "Unsupported replication type: " + replicationType);
+    }
+  }
 }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java
index 213e6c4..7befffe 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdds.client;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -78,6 +79,12 @@
   }
 
   @Override
+  @JsonIgnore
+  public String getReplication() {
+    return String.valueOf(this.replicationFactor);
+  }
+
+  @Override
   public ReplicationType getReplicationType() {
     return ReplicationType.STAND_ALONE;
   }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
index fe4ec99..2a962d7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
@@ -341,6 +341,8 @@
 
   public static final String OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY =
       "ozone.scm.container.placement.impl";
+  public static final String OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY =
+      "ozone.scm.container.placement.ec.impl";
 
   public static final String OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT =
       "ozone.scm.pipeline.owner.container.count";
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
index e2a5c8c..dd56903 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
@@ -19,6 +19,7 @@
 
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hdds.annotation.InterfaceStability;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto;
 import org.apache.hadoop.hdds.scm.DatanodeAdminError;
 import org.apache.hadoop.hdds.scm.container.ContainerReplicaInfo;
@@ -127,12 +128,14 @@
    * @param startContainerID start containerID.
    * @param count count must be {@literal >} 0.
    * @param state Container of this state will be returned.
-   * @param factor container factor.
+   * @param replicationConfig container replication Config.
    * @return a list of pipeline.
    * @throws IOException
    */
   List<ContainerInfo> listContainer(long startContainerID, int count,
-      HddsProtos.LifeCycleState state, HddsProtos.ReplicationFactor factor)
+      HddsProtos.LifeCycleState state,
+      HddsProtos.ReplicationType replicationType,
+      ReplicationConfig replicationConfig)
       throws IOException;
 
   /**
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java
index 0071924..e99b6a5 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java
@@ -26,6 +26,7 @@
 import java.util.Comparator;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
@@ -114,8 +115,8 @@
   public static ContainerInfo fromProtobuf(HddsProtos.ContainerInfoProto info) {
     ContainerInfo.Builder builder = new ContainerInfo.Builder();
     final ReplicationConfig config = ReplicationConfig
-        .fromProtoTypeAndFactor(
-            info.getReplicationType(), info.getReplicationFactor());
+        .fromProto(info.getReplicationType(), info.getReplicationFactor(),
+            info.getEcReplicationConfig());
     builder.setUsedBytes(info.getUsedBytes())
         .setNumberOfKeys(info.getNumberOfKeys())
         .setState(info.getState())
@@ -159,10 +160,12 @@
     return replicationConfig;
   }
 
+  @JsonIgnore
   public HddsProtos.ReplicationType getReplicationType() {
     return replicationConfig.getReplicationType();
   }
 
+  @JsonIgnore
   public HddsProtos.ReplicationFactor getReplicationFactor() {
     return ReplicationConfig.getLegacyFactor(replicationConfig);
   }
@@ -171,6 +174,28 @@
     return pipelineID;
   }
 
+  /**
+   * Returns the usedBytes for the container. The value returned is derived
+   * from the replicas reported from the datanodes for the container.
+   *
+   * The size of a container can change over time. For an open container we
+   * assume the size of the container will grow, and hence the value will be
+   * the maximum of the values reported from its replicas.
+   *
+   * A closed container can only reduce in size as its blocks are removed. For
+   * a closed container, the value will be the minimum of the values reported
+   * from its replicas.
+   *
+   * An EC container, is made from a group data and parity containers where the
+   * first data and all parity containers should be the same size. The other
+   * data containers can be smaller or the same size. When calculating the EC
+   * container size, we use the min / max of the first data and parity
+   * containers,ignoring the others. For EC containers, this value actually
+   * represents the size of the largest container in the container group, rather
+   * than the total space used by all containers in the group.
+   *
+   * @return bytes used in the container.
+   */
   public long getUsedBytes() {
     return usedBytes;
   }
@@ -232,10 +257,17 @@
         .setContainerID(getContainerID())
         .setDeleteTransactionId(getDeleteTransactionId())
         .setOwner(getOwner())
-        .setSequenceId(getSequenceId());
+        .setSequenceId(getSequenceId())
+        .setReplicationType(getReplicationType());
 
-    builder.setReplicationFactor(
-        ReplicationConfig.getLegacyFactor(replicationConfig));
+    if (replicationConfig instanceof ECReplicationConfig) {
+      builder.setEcReplicationConfig(((ECReplicationConfig) replicationConfig)
+          .toProto());
+    } else {
+      builder.setReplicationFactor(
+          ReplicationConfig.getLegacyFactor(replicationConfig));
+    }
+
     builder.setReplicationType(replicationConfig.getReplicationType());
 
     if (getPipelineID() != null) {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaInfo.java
index b30dff7..5a81f6b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaInfo.java
@@ -34,6 +34,7 @@
   private long sequenceId;
   private long keyCount;
   private long bytesUsed;
+  private int replicaIndex = -1;
 
   public static ContainerReplicaInfo fromProto(
       HddsProtos.SCMContainerReplicaProto proto) {
@@ -45,7 +46,9 @@
         .setPlaceOfBirth(UUID.fromString(proto.getPlaceOfBirth()))
         .setSequenceId(proto.getSequenceID())
         .setKeyCount(proto.getKeyCount())
-        .setBytesUsed(proto.getBytesUsed());
+        .setBytesUsed(proto.getBytesUsed())
+        .setReplicaIndex(
+            proto.hasReplicaIndex() ? (int)proto.getReplicaIndex() : -1);
     return builder.build();
   }
 
@@ -80,6 +83,10 @@
     return bytesUsed;
   }
 
+  public int getReplicaIndex() {
+    return replicaIndex;
+  }
+
   /**
    * Builder for ContainerReplicaInfo class.
    */
@@ -122,6 +129,11 @@
       return this;
     }
 
+    public Builder setReplicaIndex(int replicaIndex) {
+      subject.replicaIndex = replicaIndex;
+      return this;
+    }
+
     public ContainerReplicaInfo build() {
       return subject;
     }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java
index 824a1f5..026b313 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java
@@ -17,16 +17,20 @@
 
 package org.apache.hadoop.hdds.scm.container.common.helpers;
 
-
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.ozone.common.MonotonicClock;
 
+import java.time.ZoneOffset;
 import java.util.Collection;
 import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * This class contains set of dns and containers which ozone client provides
@@ -34,15 +38,24 @@
  */
 public class ExcludeList {
 
-  private final Set<DatanodeDetails> datanodes;
+  private final Map<DatanodeDetails, Long> datanodes;
   private final Set<ContainerID> containerIds;
   private final Set<PipelineID> pipelineIds;
+  private long expiryTime = 0;
+  private java.time.Clock clock;
 
 
   public ExcludeList() {
-    datanodes = new HashSet<>();
+    datanodes = new ConcurrentHashMap<>();
     containerIds = new HashSet<>();
     pipelineIds = new HashSet<>();
+    clock = new MonotonicClock(ZoneOffset.UTC);
+  }
+
+  public ExcludeList(long autoExpiryTime, java.time.Clock clock) {
+    this();
+    this.expiryTime = autoExpiryTime;
+    this.clock = clock;
   }
 
   public Set<ContainerID> getContainerIds() {
@@ -50,7 +63,23 @@
   }
 
   public Set<DatanodeDetails> getDatanodes() {
-    return datanodes;
+    Set<DatanodeDetails> dns = new HashSet<>();
+    if (expiryTime > 0) {
+      Iterator<Map.Entry<DatanodeDetails, Long>> iterator =
+          datanodes.entrySet().iterator();
+      while (iterator.hasNext()) {
+        Map.Entry<DatanodeDetails, Long> entry = iterator.next();
+        Long storedExpiryTime = entry.getValue();
+        if (clock.millis() > storedExpiryTime) {
+          iterator.remove(); // removing
+        } else {
+          dns.add(entry.getKey());
+        }
+      }
+    } else {
+      dns = datanodes.keySet();
+    }
+    return dns;
   }
 
   public void addDatanodes(Collection<DatanodeDetails> dns) {
@@ -58,7 +87,7 @@
   }
 
   public void addDatanode(DatanodeDetails dn) {
-    datanodes.add(dn);
+    datanodes.put(dn, clock.millis() + expiryTime);
   }
 
   public void addConatinerId(ContainerID containerId) {
@@ -78,9 +107,7 @@
         HddsProtos.ExcludeListProto.newBuilder();
     containerIds
         .forEach(id -> builder.addContainerIds(id.getId()));
-    datanodes.forEach(dn -> {
-      builder.addDatanodes(dn.getUuidString());
-    });
+    getDatanodes().forEach(dn -> builder.addDatanodes(dn.getUuidString()));
     pipelineIds.forEach(pipelineID -> {
       builder.addPipelineIds(pipelineID.getProtobuf());
     });
@@ -105,7 +132,7 @@
   }
 
   public boolean isEmpty() {
-    return datanodes.isEmpty() && containerIds.isEmpty() && pipelineIds
+    return getDatanodes().isEmpty() && containerIds.isEmpty() && pipelineIds
         .isEmpty();
   }
 
@@ -118,9 +145,10 @@
   @Override
   public String toString() {
     return "ExcludeList {" +
-        "datanodes = " + datanodes +
+        "datanodes = " + getDatanodes() +
         ", containerIds = " + containerIds +
         ", pipelineIds = " + pipelineIds +
         '}';
   }
+
 }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
index f5c0b62..116e767 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
@@ -23,6 +23,7 @@
 import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -34,6 +35,7 @@
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -55,6 +57,7 @@
 
   private PipelineState state;
   private Map<DatanodeDetails, Long> nodeStatus;
+  private Map<DatanodeDetails, Integer> replicaIndexes;
   // nodes with ordered distance to client
   private ThreadLocal<List<DatanodeDetails>> nodesInOrder = new ThreadLocal<>();
   // Current reported Leader for the pipeline
@@ -78,6 +81,7 @@
     this.nodeStatus = nodeStatus;
     this.creationTimestamp = Instant.now();
     this.suggestedLeaderId = suggestedLeaderId;
+    this.replicaIndexes = new HashMap<>();
   }
 
   /**
@@ -178,6 +182,21 @@
 
 
   /**
+   * Return the replica index of the specific datanode in the datanode set.
+   * <p>
+   * For non-EC case all the replication should be exactly the same,
+   * therefore the replication index can always be zero. In case of EC
+   * different Datanodes can have different data for one specific block
+   * (parity and/or data parts) therefore the replicaIndex should be
+   * different for each of the pipeline members.
+   *
+   * @param dn datanode details
+   */
+  public int getReplicaIndex(DatanodeDetails dn) {
+    return replicaIndexes.getOrDefault(dn, 0);
+  }
+
+  /**
    * Returns the leader if found else defaults to closest node.
    *
    * @return {@link DatanodeDetails}
@@ -245,6 +264,12 @@
   }
 
   public boolean isHealthy() {
+    // EC pipelines are not reported by the DN and do not have a leader. If a
+    // node goes stale or dead, EC pipelines will by closed like RATIS pipelines
+    // but at the current time there are not other health metrics for EC.
+    if (replicationConfig.getReplicationType() == ReplicationType.EC) {
+      return true;
+    }
     for (Long reportedTime : nodeStatus.values()) {
       if (reportedTime < 0) {
         return false;
@@ -263,20 +288,30 @@
 
   public HddsProtos.Pipeline getProtobufMessage(int clientVersion)
       throws UnknownPipelineStateException {
+
     List<HddsProtos.DatanodeDetailsProto> members = new ArrayList<>();
+    List<Integer> memberReplicaIndexes = new ArrayList<>();
+
     for (DatanodeDetails dn : nodeStatus.keySet()) {
       members.add(dn.toProto(clientVersion));
+      memberReplicaIndexes.add(replicaIndexes.getOrDefault(dn, 0));
     }
 
     HddsProtos.Pipeline.Builder builder = HddsProtos.Pipeline.newBuilder()
         .setId(id.getProtobuf())
         .setType(replicationConfig.getReplicationType())
-        .setFactor(ReplicationConfig.getLegacyFactor(replicationConfig))
         .setState(PipelineState.getProtobuf(state))
         .setLeaderID(leaderId != null ? leaderId.toString() : "")
         .setCreationTimeStamp(creationTimestamp.toEpochMilli())
-        .addAllMembers(members);
+        .addAllMembers(members)
+        .addAllMemberReplicaIndexes(memberReplicaIndexes);
 
+    if (replicationConfig instanceof ECReplicationConfig) {
+      builder.setEcReplicationConfig(((ECReplicationConfig) replicationConfig)
+          .toProto());
+    } else {
+      builder.setFactor(ReplicationConfig.getLegacyFactor(replicationConfig));
+    }
     if (leaderId != null) {
       HddsProtos.UUID uuid128 = HddsProtos.UUID.newBuilder()
           .setMostSigBits(leaderId.getMostSignificantBits())
@@ -317,9 +352,16 @@
       throws UnknownPipelineStateException {
     Preconditions.checkNotNull(pipeline, "Pipeline is null");
 
-    List<DatanodeDetails> nodes = new ArrayList<>();
+    Map<DatanodeDetails, Integer> nodes = new LinkedHashMap<>();
+    int index = 0;
+    int repIndexListLength = pipeline.getMemberReplicaIndexesCount();
     for (DatanodeDetailsProto member : pipeline.getMembersList()) {
-      nodes.add(DatanodeDetails.getFromProtoBuf(member));
+      int repIndex = 0;
+      if (index < repIndexListLength) {
+        repIndex = pipeline.getMemberReplicaIndexes(index);
+      }
+      nodes.put(DatanodeDetails.getFromProtoBuf(member), repIndex);
+      index++;
     }
     UUID leaderId = null;
     if (pipeline.hasLeaderID128()) {
@@ -338,11 +380,13 @@
     }
 
     final ReplicationConfig config = ReplicationConfig
-        .fromProtoTypeAndFactor(pipeline.getType(), pipeline.getFactor());
+        .fromProto(pipeline.getType(), pipeline.getFactor(),
+            pipeline.getEcReplicationConfig());
     return new Builder().setId(PipelineID.getFromProtobuf(pipeline.getId()))
         .setReplicationConfig(config)
         .setState(PipelineState.fromProtobuf(pipeline.getState()))
-        .setNodes(nodes)
+        .setNodes(new ArrayList<>(nodes.keySet()))
+        .setReplicaIndexes(nodes)
         .setLeaderId(leaderId)
         .setSuggestedLeaderId(suggestedLeaderId)
         .setNodesInOrder(pipeline.getMemberOrdersList())
@@ -401,6 +445,10 @@
     return new Builder(pipeline);
   }
 
+  private void setReplicaIndexes(Map<DatanodeDetails, Integer> replicaIndexes) {
+    this.replicaIndexes = replicaIndexes;
+  }
+
   /**
    * Builder class for Pipeline.
    */
@@ -414,6 +462,7 @@
     private UUID leaderId = null;
     private Instant creationTimestamp = null;
     private UUID suggestedLeaderId = null;
+    private Map<DatanodeDetails, Integer> replicaIndexes = new HashMap<>();
 
     public Builder() { }
 
@@ -426,6 +475,15 @@
       this.leaderId = pipeline.getLeaderId();
       this.creationTimestamp = pipeline.getCreationTimestamp();
       this.suggestedLeaderId = pipeline.getSuggestedLeaderId();
+      this.replicaIndexes = new HashMap<>();
+      if (nodeStatus != null) {
+        for (DatanodeDetails dn : nodeStatus.keySet()) {
+          int index = pipeline.getReplicaIndex(dn);
+          if (index > 0) {
+            replicaIndexes.put(dn, index);
+          }
+        }
+      }
     }
 
     public Builder setId(PipelineID id1) {
@@ -447,7 +505,6 @@
       this.leaderId = leaderId1;
       return this;
     }
-
     public Builder setNodes(List<DatanodeDetails> nodes) {
       this.nodeStatus = new LinkedHashMap<>();
       nodes.forEach(node -> nodeStatus.put(node, -1L));
@@ -469,6 +526,12 @@
       return this;
     }
 
+
+    public Builder setReplicaIndexes(Map<DatanodeDetails, Integer> indexes) {
+      this.replicaIndexes = indexes;
+      return this;
+    }
+
     public Pipeline build() {
       Preconditions.checkNotNull(id);
       Preconditions.checkNotNull(replicationConfig);
@@ -483,6 +546,8 @@
         pipeline.setCreationTimestamp(creationTimestamp);
       }
 
+      pipeline.setReplicaIndexes(replicaIndexes);
+
       if (nodeOrder != null && !nodeOrder.isEmpty()) {
         // This branch is for build from ProtoBuf
         List<DatanodeDetails> nodesWithOrder = new ArrayList<>();
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
index cd94100..d915a20 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdds.scm.protocol;
 
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type;
@@ -184,6 +185,28 @@
       int count, HddsProtos.LifeCycleState state,
       HddsProtos.ReplicationFactor factor) throws IOException;
 
+
+  /**
+   * Ask SCM for a list of containers with a range of container ID, state
+   * and replication config, and the limit of count.
+   * The containers are returned from startID (exclusive), and
+   * filtered by state and replication config. The returned list is limited to
+   * count entries.
+   *
+   * @param startContainerID start container ID.
+   * @param count count, if count {@literal <} 0, the max size is unlimited.(
+   *              Usually the count will be replace with a very big
+   *              value instead of being unlimited in case the db is very big)
+   * @param state Container with this state will be returned.
+   * @param replicationConfig Replication config for the containers
+   * @return a list of container.
+   * @throws IOException
+   */
+  List<ContainerInfo> listContainer(long startContainerID,
+      int count, HddsProtos.LifeCycleState state,
+      HddsProtos.ReplicationType replicationType,
+      ReplicationConfig replicationConfig) throws IOException;
+
   /**
    * Deletes a container in SCM.
    *
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index 7f2d2a8..2597870 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -282,17 +282,27 @@
    */
   public static XceiverClientReply writeChunkAsync(
       XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID,
-      ByteString data, Token<? extends TokenIdentifier> token)
+      ByteString data, Token<? extends TokenIdentifier> token,
+      int replicationIndex
+  )
       throws IOException, ExecutionException, InterruptedException {
     WriteChunkRequestProto.Builder writeChunkRequest =
         WriteChunkRequestProto.newBuilder()
-            .setBlockID(blockID.getDatanodeBlockIDProtobuf())
-            .setChunkData(chunk).setData(data);
+            .setBlockID(DatanodeBlockID.newBuilder()
+                .setContainerID(blockID.getContainerID())
+                .setLocalID(blockID.getLocalID())
+                .setBlockCommitSequenceId(blockID.getBlockCommitSequenceId())
+                .setReplicaIndex(replicationIndex)
+                .build())
+            .setChunkData(chunk)
+            .setData(data);
     String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
     ContainerCommandRequestProto.Builder builder =
-        ContainerCommandRequestProto.newBuilder().setCmdType(Type.WriteChunk)
+        ContainerCommandRequestProto.newBuilder()
+            .setCmdType(Type.WriteChunk)
             .setContainerID(blockID.getContainerID())
-            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+            .setDatanodeUuid(id)
+            .setWriteChunk(writeChunkRequest);
     if (token != null) {
       builder.setEncodedToken(token.encodeToUrlString());
     }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java
index 2bc1a67..4217055 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java
@@ -31,7 +31,9 @@
   INITIAL_VERSION(0, "Initial Layout Version"),
   DATANODE_SCHEMA_V2(1, "Datanode RocksDB Schema Version 2 (with column " +
       "families)"),
-  SCM_HA(2, "Storage Container Manager HA");
+  SCM_HA(2, "Storage Container Manager HA"),
+  ERASURE_CODED_STORAGE_SUPPORT(3, "Ozone version with built in support for"
+      + " Erasure Coded block data storage.");
 
   //////////////////////////////  //////////////////////////////
 
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java
index ec00ca2..b8e2542 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java
@@ -35,6 +35,9 @@
   VERSION_HANDLES_UNKNOWN_DN_PORTS(1,
       "Client version that handles the REPLICATION port in DatanodeDetails."),
 
+  ERASURE_CODING_SUPPORT(2, "This client version has support for Erasure"
+      + " Coding."),
+
   FUTURE_VERSION(-1, "Used internally when the server side is older and an"
       + " unknown client version has arrived from the client.");
 
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index d5794e5..699d732 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -115,6 +115,11 @@
       "ozone.scm.block.size";
   public static final String OZONE_SCM_BLOCK_SIZE_DEFAULT = "256MB";
 
+  public static final String OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES =
+      "ozone.client.max.ec.stripe.write.retries";
+  public static final String OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES_DEFAULT =
+      "10";
+
   /**
    * Ozone administrator users delimited by comma.
    * If not set, only the user who launches an ozone service will be the
@@ -455,6 +460,13 @@
   public static final String OZONE_CLIENT_REQUIRED_OM_VERSION_MIN_DEFAULT =
       OzoneManagerVersion.S3G_PERSISTENT_CONNECTIONS.name();
 
+  public static final String
+      OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_MS =
+      "ozone.client.bucket.replication.config.refresh.time.ms";
+  public static final long
+      OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_DEFAULT_MS =
+      300 * 1000;
+
   public static final String OZONE_AUDIT_LOG_DEBUG_CMD_LIST_OMAUDIT =
       "ozone.audit.log.debug.cmd.list.omaudit";
   /**
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 0430283..a9198bd 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -261,6 +261,7 @@
   public static final String ORIGIN_PIPELINE_ID = "originPipelineId";
   public static final String ORIGIN_NODE_ID = "originNodeId";
   public static final String SCHEMA_VERSION = "schemaVersion";
+  public static final String REPLICA_INDEX = "replicaIndex";
 
   // Supported .container datanode schema versions.
   // Since containers in older schema versions are currently not reformatted to
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java
index ff3d112..c6872f2 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java
@@ -32,6 +32,8 @@
   DEFAULT_VERSION(0, "Initial version"),
   S3G_PERSISTENT_CONNECTIONS(1,
       "New S3G persistent connection support is present in OM."),
+  ERASURE_CODED_STORAGE_SUPPORT(2, "OzoneManager version that supports"
+      + "ECReplicationConfig"),
 
   FUTURE_VERSION(-1, "Used internally in the client when the server side is "
       + " newer and an unknown server version has arrived to the client.");
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 703e3c5..aaa7859 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -811,6 +811,20 @@
     </description>
   </property>
   <property>
+    <name>ozone.scm.container.placement.ec.impl</name>
+    <value>
+      org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRackScatter
+    </value>
+    <tag>OZONE, MANAGEMENT</tag>
+    <description>
+      The full name of class which implements
+      org.apache.hadoop.hdds.scm.PlacementPolicy.
+      The class decides which datanode will be used to host the container replica in EC mode. If not set,
+      org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom will be used as default
+      value.
+    </description>
+  </property>
+  <property>
     <name>ozone.scm.pipeline.owner.container.count</name>
     <value>3</value>
     <tag>OZONE, SCM, PIPELINE</tag>
@@ -1145,26 +1159,41 @@
   </property>
 
   <property>
-    <name>ozone.replication</name>
+    <name>ozone.server.default.replication</name>
     <value>3</value>
-    <tag>OZONE, CLIENT</tag>
+    <tag>OZONE</tag>
     <description>
       Default replication value. The actual number of replications can be
       specified when writing the key. The default is used if replication
-      is not specified. Supported values: 1 and 3.
+      is not specified when creating key or no default replication set at
+      bucket. Supported values: 1, 3 and EC_3_2.
     </description>
   </property>
 
   <property>
-    <name>ozone.replication.type</name>
+    <name>ozone.server.default.replication.type</name>
     <value>RATIS</value>
-    <tag>OZONE, CLIENT</tag>
+    <tag>OZONE</tag>
     <description>
       Default replication type to be used while writing key into ozone. The
       value can be specified when writing the key, default is used when
-      nothing is specified. Supported values: RATIS, STAND_ALONE and CHAINED.
+      nothing is specified when creating key or no default value set at bucket.
+      Supported values: RATIS, STAND_ALONE, CHAINED and EC.
     </description>
   </property>
+
+  <property>
+    <name>ozone.client.bucket.replication.config.refresh.time.ms</name>
+    <value>30000</value>
+    <tag>OZONE</tag>
+    <description>
+      Default time period to refresh the bucket replication config in o3fs
+      clients. Until the bucket replication config refreshed, client will
+      continue to use existing replication config irrespective of whether bucket
+      replication config updated at OM or not.
+    </description>
+  </property>
+
   <property>
     <name>hdds.container.close.threshold</name>
     <value>0.9f</value>
@@ -3090,6 +3119,18 @@
   </property>
 
   <property>
+    <name>ozone.client.max.ec.stripe.write.retries</name>
+    <value>10</value>
+    <tag>CLIENT</tag>
+    <description>
+      When EC stripe write failed, client will request to allocate new block group and write the failed stripe into new
+      block group. If the same stripe failure continued in newly acquired block group also, then it will retry by
+      requesting to allocate new block group again. This configuration is used to limit these number of retries. By
+      default the number of retries are 10.
+    </description>
+  </property>
+
+  <property>
     <name>ozone.audit.log.debug.cmd.list.omaudit</name>
     <value></value>
     <tag>OM</tag>
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestECReplicationConfig.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestECReplicationConfig.java
new file mode 100644
index 0000000..9440a6e
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestECReplicationConfig.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.client;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec.RS;
+import static org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec.XOR;
+import static org.junit.Assert.fail;
+
+/**
+ * Unit test for ECReplicationConfig.
+ */
+public class TestECReplicationConfig {
+
+  @Test
+  public void testSuccessfulStringParsing() {
+    Map<String, ECReplicationConfig> valid = new HashMap();
+    valid.put("rs-3-2-1024", new ECReplicationConfig(3, 2, RS, 1024));
+    valid.put("RS-3-2-1024", new ECReplicationConfig(3, 2, RS, 1024));
+    valid.put("rs-3-2-1024k", new ECReplicationConfig(3, 2, RS, 1024 * 1024));
+    valid.put("rs-3-2-1024K", new ECReplicationConfig(3, 2, RS, 1024 * 1024));
+    valid.put("xor-10-4-1", new ECReplicationConfig(10, 4, XOR, 1));
+    valid.put("XOR-6-3-12345", new ECReplicationConfig(6, 3, XOR, 12345));
+
+    for (Map.Entry<String, ECReplicationConfig> e : valid.entrySet()) {
+      ECReplicationConfig ec = new ECReplicationConfig(e.getKey());
+      Assert.assertEquals(e.getValue().getData(), ec.getData());
+      Assert.assertEquals(e.getValue().getParity(), ec.getParity());
+      Assert.assertEquals(e.getValue().getCodec(), ec.getCodec());
+      Assert.assertEquals(e.getValue().getEcChunkSize(), ec.getEcChunkSize());
+    }
+  }
+
+  @Test
+  public void testUnsuccessfulStringParsing() {
+    String[] invalid = {
+        "3-2-1024",
+        "rss-3-2-1024",
+        "rs-3-0-1024",
+        "rs-3-2-0k",
+        "rs-3-2",
+        "x3-2"
+    };
+    for (String s : invalid) {
+      try {
+        new ECReplicationConfig(s);
+        fail(s + " should not parse correctly");
+      } catch (IllegalArgumentException e) {
+        // ignore, this expected
+      }
+    }
+  }
+
+
+  @Test
+  public void testSerializeToProtoAndBack() {
+    ECReplicationConfig orig = new ECReplicationConfig(6, 3,
+        ECReplicationConfig.EcCodec.XOR, 1024);
+
+    HddsProtos.ECReplicationConfig proto = orig.toProto();
+
+    ECReplicationConfig recovered = new ECReplicationConfig(proto);
+    Assert.assertEquals(orig.getData(), recovered.getData());
+    Assert.assertEquals(orig.getParity(), recovered.getParity());
+    Assert.assertEquals(orig.getCodec(), recovered.getCodec());
+    Assert.assertEquals(orig.getEcChunkSize(), recovered.getEcChunkSize());
+    Assert.assertTrue(orig.equals(recovered));
+  }
+
+}
\ No newline at end of file
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestReplicationConfig.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestReplicationConfig.java
index 1315ad5..ed8c8f7 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestReplicationConfig.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestReplicationConfig.java
@@ -17,10 +17,14 @@
  */
 package org.apache.hadoop.hdds.client;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -36,8 +40,11 @@
 @RunWith(Parameterized.class)
 public class TestReplicationConfig {
 
+  private static final int MB = 1024 * 1024;
+  private static final int KB = 1024;
+
   @SuppressWarnings("checkstyle:VisibilityModifier")
-  @Parameterized.Parameter()
+  @Parameterized.Parameter
   public String type;
 
   @SuppressWarnings("checkstyle:VisibilityModifier")
@@ -46,24 +53,49 @@
 
   @SuppressWarnings("checkstyle:VisibilityModifier")
   @Parameterized.Parameter(2)
+  public String codec;
+
+  @SuppressWarnings("checkstyle:VisibilityModifier")
+  @Parameterized.Parameter(3)
+  public int data;
+
+  @SuppressWarnings("checkstyle:VisibilityModifier")
+  @Parameterized.Parameter(4)
+  public int parity;
+
+  @SuppressWarnings("checkstyle:VisibilityModifier")
+  @Parameterized.Parameter(5)
+  public int chunkSize;
+
+  @SuppressWarnings("checkstyle:VisibilityModifier")
+  @Parameterized.Parameter(6)
   public Class<?> replicationConfigClass;
 
+  //NOTE: if a new cunckSize is used/added in the parameters other than KB or MB
+  // please revisit the method createECDescriptor, to handle the new chunkSize.
   @Parameterized.Parameters(name = "{0}/{1}")
   public static Object[][] parameters() {
     return new Object[][] {
-        {"RATIS", "ONE", RatisReplicationConfig.class },
-        {"RATIS", "THREE", RatisReplicationConfig.class},
-        {"STAND_ALONE", "ONE", StandaloneReplicationConfig.class},
-        {"STAND_ALONE", "THREE", StandaloneReplicationConfig.class}
+        {"RATIS", "ONE", null, 0, 0, 0, RatisReplicationConfig.class },
+        {"RATIS", "THREE", null, 0, 0, 0, RatisReplicationConfig.class},
+        {"STAND_ALONE", "ONE", null, 0, 0, 0,
+            StandaloneReplicationConfig.class},
+        {"STAND_ALONE", "THREE", null, 0, 0, 0,
+            StandaloneReplicationConfig.class},
+        {"EC", "RS-3-2-1024K", "RS", 3, 2, MB, ECReplicationConfig.class},
+        {"EC", "RS-3-2-1024", "RS", 3, 2, KB, ECReplicationConfig.class},
+        {"EC", "RS-6-3-1024K", "RS", 6, 3, MB, ECReplicationConfig.class},
+        {"EC", "RS-6-3-1024", "RS", 6, 3, KB, ECReplicationConfig.class},
+        {"EC", "RS-10-4-1024K", "RS", 10, 4, MB, ECReplicationConfig.class},
+        {"EC", "RS-10-4-1024", "RS", 10, 4, KB, ECReplicationConfig.class},
     };
   }
 
   @Test
-  public void testGetDefaultShouldCreateReplicationConfigFromDefaultConf() {
+  public void testGetDefaultShouldReturnRatisThreeIfNotSetClientSide() {
     OzoneConfiguration conf = new OzoneConfiguration();
 
     ReplicationConfig replicationConfig = ReplicationConfig.getDefault(conf);
-
     validate(replicationConfig,
         org.apache.hadoop.hdds.client.ReplicationType.RATIS,
         org.apache.hadoop.hdds.client.ReplicationFactor.THREE,
@@ -71,7 +103,8 @@
   }
 
   @Test
-  public void testGetDefaultShouldCreateReplicationConfFromCustomConfValues() {
+  public void testGetDefaultShouldCreateReplicationConfFromConfValues() {
+    assumeRatisOrStandaloneType();
     OzoneConfiguration conf = new OzoneConfiguration();
     conf.set(OZONE_REPLICATION_TYPE, type);
     conf.set(OZONE_REPLICATION, factor);
@@ -84,7 +117,22 @@
   }
 
   @Test
+  public void testGetDefaultShouldCreateECReplicationConfFromConfValues() {
+    assumeECType();
+
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.set(OZONE_REPLICATION_TYPE, type);
+    conf.set(OZONE_REPLICATION, ecDescriptor());
+
+    ReplicationConfig replicationConfig = ReplicationConfig.getDefault(conf);
+
+    validate(replicationConfig,
+        EcCodec.valueOf(codec), data, parity, chunkSize);
+  }
+
+  @Test
   public void deserialize() {
+    assumeRatisOrStandaloneType();
     final ReplicationConfig replicationConfig =
         ReplicationConfig.fromProtoTypeAndFactor(
             ReplicationType.valueOf(type),
@@ -96,7 +144,53 @@
   }
 
   @Test
+  public void deserializeEC() {
+    assumeECType();
+    HddsProtos.ECReplicationConfig proto =
+        HddsProtos.ECReplicationConfig.newBuilder()
+            .setCodec(codec)
+            .setData(data)
+            .setParity(parity)
+            .setEcChunkSize(chunkSize)
+            .build();
+
+    ReplicationConfig config = ReplicationConfig
+        .fromProto(ReplicationType.EC, null, proto);
+
+    validate(config, EcCodec.valueOf(codec), data, parity, chunkSize);
+  }
+
+  @Test
+  public void testECReplicationConfigGetReplication() {
+    assumeECType();
+    HddsProtos.ECReplicationConfig proto =
+        HddsProtos.ECReplicationConfig.newBuilder().setCodec(codec)
+            .setData(data).setParity(parity).setEcChunkSize(chunkSize).build();
+
+    ReplicationConfig config =
+        ReplicationConfig.fromProto(ReplicationType.EC, null, proto);
+
+    Assert.assertEquals(EcCodec.valueOf(
+        codec) + ECReplicationConfig.EC_REPLICATION_PARAMS_DELIMITER
+            + data + ECReplicationConfig.EC_REPLICATION_PARAMS_DELIMITER
+            + parity + ECReplicationConfig.EC_REPLICATION_PARAMS_DELIMITER
+            + chunkSize, config.getReplication());
+  }
+
+  @Test
+  public void testReplicationConfigGetReplication() {
+    assumeRatisOrStandaloneType();
+    final ReplicationConfig replicationConfig = ReplicationConfig
+        .fromTypeAndFactor(
+            org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
+            org.apache.hadoop.hdds.client.ReplicationFactor.valueOf(factor));
+
+    Assert.assertEquals(factor, replicationConfig.getReplication());
+  }
+
+  @Test
   public void fromJavaObjects() {
+    assumeRatisOrStandaloneType();
     final ReplicationConfig replicationConfig =
         ReplicationConfig.fromTypeAndFactor(
             org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
@@ -109,6 +203,7 @@
 
   @Test
   public void testParseFromTypeAndFactorAsString() {
+    assumeRatisOrStandaloneType();
     ConfigurationSource conf = new OzoneConfiguration();
     ReplicationConfig replicationConfig = ReplicationConfig.parse(
         org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
@@ -121,10 +216,11 @@
 
   @Test
   public void testParseFromTypeAndFactorAsStringifiedInteger() {
+    assumeRatisOrStandaloneType();
     ConfigurationSource conf = new OzoneConfiguration();
     String f =
-        factor == "ONE" ? "1"
-            : factor == "THREE" ? "3"
+        factor.equals("ONE") ? "1"
+            : factor.equals("THREE") ? "3"
             : "Test adjustment needed!";
 
     ReplicationConfig replicationConfig = ReplicationConfig.parse(
@@ -137,7 +233,27 @@
   }
 
   @Test
+  public void testParseECReplicationConfigFromString() {
+    assumeECType();
+
+    ConfigurationSource conf = new OzoneConfiguration();
+    ReplicationConfig repConfig = ReplicationConfig.parse(
+        org.apache.hadoop.hdds.client.ReplicationType.EC, ecDescriptor(), conf);
+
+    validate(repConfig, EcCodec.valueOf(codec), data, parity, chunkSize);
+  }
+
+  /**
+   * The adjustReplication is a method that is used by RootedOzoneFileSystem
+   * to adjust the bucket's default replication config if needed.
+   *
+   * As we define, if the bucket's default replication configuration is RATIS
+   * or STAND_ALONE, then replica count can be adjusted with the replication
+   * factor.
+   */
+  @Test
   public void testAdjustReplication() {
+    assumeRatisOrStandaloneType();
     ConfigurationSource conf = new OzoneConfiguration();
     ReplicationConfig replicationConfig = ReplicationConfig.parse(
         org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
@@ -155,6 +271,30 @@
   }
 
   /**
+   * The adjustReplication is a method that is used by RootedOzoneFileSystem
+   * to adjust the bucket's default replication config if needed.
+   *
+   * As we define, if the bucket's default replication configuration is EC,
+   * then the client can not adjust the configuration via the replication
+   * factor.
+   */
+  @Test
+  public void testAdjustECReplication() {
+    assumeECType();
+    ConfigurationSource conf = new OzoneConfiguration();
+    ReplicationConfig replicationConfig = ReplicationConfig.parse(
+        org.apache.hadoop.hdds.client.ReplicationType.EC, ecDescriptor(), conf);
+
+    validate(
+        ReplicationConfig.adjustReplication(replicationConfig, (short) 3, conf),
+        EcCodec.valueOf(codec), data, parity, chunkSize);
+
+    validate(
+        ReplicationConfig.adjustReplication(replicationConfig, (short) 1, conf),
+        EcCodec.valueOf(codec), data, parity, chunkSize);
+  }
+
+  /**
    * This is a bit of a tricky test in the parametrized environment.
    * The goal is to ensure that the following methods do validation while
    * creating the ReplicationConfig: getDefault, adjustReplication, parse.
@@ -166,17 +306,21 @@
    */
   @Test
   public void testValidationBasedOnConfig() {
+    assumeRatisOrStandaloneType();
     OzoneConfiguration conf = new OzoneConfiguration();
     conf.set(OZONE_REPLICATION + ".allowed-configs",
         "^STANDALONE/ONE|RATIS/THREE$");
     conf.set(OZONE_REPLICATION, factor);
     conf.set(OZONE_REPLICATION_TYPE, type);
 
+    // in case of allowed configurations
     if ((type.equals("RATIS") && factor.equals("THREE"))
         || (type.equals("STAND_ALONE") && factor.equals("ONE"))) {
       ReplicationConfig replicationConfig = ReplicationConfig.parse(
           org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
           factor, conf);
+      // check if adjust throws exception when adjusting to a config that is
+      // not allowed
       if (type.equals("RATIS")) {
         assertThrows(IllegalArgumentException.class,
             () -> ReplicationConfig
@@ -186,27 +330,33 @@
             () -> ReplicationConfig
                 .adjustReplication(replicationConfig, (short) 3, conf));
       }
-      ReplicationConfig.fromTypeAndFactor(
-          org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
-          org.apache.hadoop.hdds.client.ReplicationFactor.valueOf(factor));
-      ReplicationConfig.fromProtoTypeAndFactor(
-          ReplicationType.valueOf(type), ReplicationFactor.valueOf(factor));
       ReplicationConfig.getDefault(conf);
     } else {
+      // parse should fail in case of a configuration that is not allowed.
       assertThrows(IllegalArgumentException.class,
           () -> ReplicationConfig.parse(
               org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
               factor, conf));
+      // default can not be a configuration that is not allowed.
       assertThrows(IllegalArgumentException.class,
           () -> ReplicationConfig.getDefault(conf));
     }
+
+    // From proto and java objects, we need to be able to create replication
+    // configs even though they are not allowed, as there might have been
+    // keys, that were created earlier when the config was allowed.
+    ReplicationConfig.fromTypeAndFactor(
+        org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
+        org.apache.hadoop.hdds.client.ReplicationFactor.valueOf(factor));
+    ReplicationConfig.fromProtoTypeAndFactor(
+        ReplicationType.valueOf(type), ReplicationFactor.valueOf(factor));
+
     // CHAINED replication type is not supported by ReplicationConfig.
     assertThrows(RuntimeException.class,
         () -> ReplicationConfig.parse(
             org.apache.hadoop.hdds.client.ReplicationType.CHAINED, "", conf));
   }
 
-
   private void validate(ReplicationConfig replicationConfig,
       org.apache.hadoop.hdds.client.ReplicationType expectedType,
       org.apache.hadoop.hdds.client.ReplicationFactor expectedFactor) {
@@ -232,4 +382,36 @@
         ((ReplicatedReplicationConfig) replicationConfig)
             .getReplicationFactor().name());
   }
+
+  private void validate(ReplicationConfig replicationConfig,
+      EcCodec expectedCodec,
+      int expectedData, int expectedParity, int expectedChunkSize) {
+
+    assertEquals(ECReplicationConfig.class, replicationConfig.getClass());
+    assertEquals(ReplicationType.EC, replicationConfig.getReplicationType());
+
+    ECReplicationConfig ecReplicationConfig =
+        (ECReplicationConfig) replicationConfig;
+
+    assertEquals(expectedCodec, ecReplicationConfig.getCodec());
+    assertEquals(expectedData, ecReplicationConfig.getData());
+    assertEquals(expectedParity, ecReplicationConfig.getParity());
+    assertEquals(expectedChunkSize, ecReplicationConfig.getEcChunkSize());
+
+    assertEquals(expectedData + expectedParity,
+        replicationConfig.getRequiredNodes());
+  }
+
+  private String ecDescriptor() {
+    return codec.toUpperCase() + "-" + data + "-" + parity + "-" +
+        (chunkSize == MB ? "1024K" : "1024");
+  }
+
+  private void assumeRatisOrStandaloneType() {
+    Assume.assumeTrue(type.equals("RATIS") || type.equals("STAND_ALONE"));
+  }
+
+  private void assumeECType() {
+    Assume.assumeTrue(type.equals("EC"));
+  }
 }
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java
new file mode 100644
index 0000000..9ec19bf
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java
@@ -0,0 +1,86 @@
+/*
+ * 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.hdds.scm.container;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Tests for the ContainerInfo class.
+ */
+
+public class TestContainerInfo {
+
+  @Test
+  public void getProtobufMessageEC() throws IOException {
+    ContainerInfo container =
+        createContainerInfo(RatisReplicationConfig.getInstance(THREE));
+    HddsProtos.ContainerInfoProto proto = container.getProtobuf();
+
+    // No EC Config
+    Assert.assertFalse(proto.hasEcReplicationConfig());
+    Assert.assertEquals(THREE, proto.getReplicationFactor());
+    Assert.assertEquals(RATIS, proto.getReplicationType());
+
+    // Reconstruct object from Proto
+    ContainerInfo recovered = ContainerInfo.fromProtobuf(proto);
+    Assert.assertEquals(RATIS, recovered.getReplicationType());
+    Assert.assertTrue(
+        recovered.getReplicationConfig() instanceof RatisReplicationConfig);
+
+    // EC Config
+    container = createContainerInfo(new ECReplicationConfig(3, 2));
+    proto = container.getProtobuf();
+
+    Assert.assertEquals(3, proto.getEcReplicationConfig().getData());
+    Assert.assertEquals(2, proto.getEcReplicationConfig().getParity());
+    Assert.assertFalse(proto.hasReplicationFactor());
+    Assert.assertEquals(EC, proto.getReplicationType());
+
+    // Reconstruct object from Proto
+    recovered = ContainerInfo.fromProtobuf(proto);
+    Assert.assertEquals(EC, recovered.getReplicationType());
+    Assert.assertTrue(
+        recovered.getReplicationConfig() instanceof ECReplicationConfig);
+    ECReplicationConfig config =
+        (ECReplicationConfig)recovered.getReplicationConfig();
+    Assert.assertEquals(3, config.getData());
+    Assert.assertEquals(2, config.getParity());
+  }
+
+  private ContainerInfo createContainerInfo(ReplicationConfig repConfig) {
+    ContainerInfo.Builder builder = new ContainerInfo.Builder();
+    builder.setContainerID(1234)
+        .setReplicationConfig(repConfig)
+        .setPipelineID(PipelineID.randomId())
+        .setState(HddsProtos.LifeCycleState.OPEN)
+        .setOwner("scm");
+    return builder.build();
+  }
+}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReplicaInfo.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReplicaInfo.java
index 195baca..4f63ea7 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReplicaInfo.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReplicaInfo.java
@@ -55,5 +55,36 @@
         proto.getDatanodeDetails()), info.getDatanodeDetails());
     Assert.assertEquals(proto.getSequenceID(), info.getSequenceId());
     Assert.assertEquals(proto.getState(), info.getState());
+    // If replicaIndex is not in the proto, then -1 should be returned
+    Assert.assertEquals(-1, info.getReplicaIndex());
+  }
+
+  @Test
+  public void testObjectCreatedFromProtoWithReplicaIndedx() {
+    HddsProtos.SCMContainerReplicaProto proto =
+        HddsProtos.SCMContainerReplicaProto.newBuilder()
+            .setKeyCount(10)
+            .setBytesUsed(12345)
+            .setContainerID(567)
+            .setPlaceOfBirth(UUID.randomUUID().toString())
+            .setSequenceID(5)
+            .setDatanodeDetails(MockDatanodeDetails.randomDatanodeDetails()
+                .getProtoBufMessage())
+            .setState("OPEN")
+            .setReplicaIndex(4)
+            .build();
+
+    ContainerReplicaInfo info = ContainerReplicaInfo.fromProto(proto);
+
+    Assert.assertEquals(proto.getContainerID(), info.getContainerID());
+    Assert.assertEquals(proto.getBytesUsed(), info.getBytesUsed());
+    Assert.assertEquals(proto.getKeyCount(), info.getKeyCount());
+    Assert.assertEquals(proto.getPlaceOfBirth(),
+        info.getPlaceOfBirth().toString());
+    Assert.assertEquals(DatanodeDetails.getFromProtoBuf(
+        proto.getDatanodeDetails()), info.getDatanodeDetails());
+    Assert.assertEquals(proto.getSequenceID(), info.getSequenceId());
+    Assert.assertEquals(proto.getState(), info.getState());
+    Assert.assertEquals(4, info.getReplicaIndex());
   }
 }
\ No newline at end of file
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java
new file mode 100644
index 0000000..d3fb49b
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.common.helpers;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.ozone.test.TestClock;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.util.UUID;
+
+/**
+ * Tests the exclude nodes list behavior at client.
+ */
+public class TestExcludeList {
+  private TestClock clock = new TestClock(Instant.now(), ZoneOffset.UTC);
+
+  @Test
+  public void excludeNodesShouldBeCleanedBasedOnGivenTime() {
+    ExcludeList list = new ExcludeList(10, clock);
+    list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
+        .setIpAddress("127.0.0.1").setHostName("localhost").addPort(
+            DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+        .build());
+    Assert.assertTrue(list.getDatanodes().size() == 1);
+    clock.fastForward(11);
+    Assert.assertTrue(list.getDatanodes().size() == 0);
+    list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
+        .setIpAddress("127.0.0.2").setHostName("localhost").addPort(
+            DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+        .build());
+    list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
+        .setIpAddress("127.0.0.3").setHostName("localhost").addPort(
+            DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+        .build());
+    Assert.assertTrue(list.getDatanodes().size() == 2);
+  }
+
+  @Test
+  public void excludeNodeShouldNotBeCleanedIfExpiryTimeIsZero() {
+    ExcludeList list = new ExcludeList(0, clock);
+    list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
+        .setIpAddress("127.0.0.1").setHostName("localhost").addPort(
+            DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+        .build());
+    Assert.assertTrue(list.getDatanodes().size() == 1);
+    clock.fastForward(1);
+    Assert.assertTrue(list.getDatanodes().size() == 1);
+  }
+}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java
index 2f459fb..8193747 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
  Test cases for SCM container client classes.
  */
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipeline.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipeline.java
index 62917de..dc28102 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipeline.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipeline.java
@@ -19,9 +19,12 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -87,6 +90,33 @@
         .build();
   }
 
+  public static Pipeline createEcPipeline() {
+    return createEcPipeline(new ECReplicationConfig(3, 2));
+  }
+
+  public static Pipeline createEcPipeline(ECReplicationConfig repConfig) {
+
+    List<DatanodeDetails> nodes = new ArrayList<>();
+    for (int i = 0; i < repConfig.getRequiredNodes(); i++) {
+      nodes.add(MockDatanodeDetails.randomDatanodeDetails());
+    }
+    Map<DatanodeDetails, Integer> nodeIndexes = new HashMap<>();
+
+    int index = nodes.size() - 1;
+    for (DatanodeDetails dn : nodes) {
+      nodeIndexes.put(dn, index);
+      index--;
+    }
+
+    return Pipeline.newBuilder()
+        .setState(Pipeline.PipelineState.OPEN)
+        .setId(PipelineID.randomId())
+        .setReplicationConfig(repConfig)
+        .setNodes(nodes)
+        .setReplicaIndexes(nodeIndexes)
+        .build();
+  }
+
   private MockPipeline() {
     throw new UnsupportedOperationException("no instances");
   }
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipeline.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipeline.java
index 20e8283..b0f9875 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipeline.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipeline.java
@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.hdds.scm.pipeline;
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -49,4 +51,65 @@
       assertPorts(dn, ALL_PORTS);
     }
   }
+
+  @Test
+  public void getProtobufMessageEC() throws IOException {
+    Pipeline subject = MockPipeline.createPipeline(3);
+
+    //when EC config is empty/null
+    HddsProtos.Pipeline protobufMessage = subject.getProtobufMessage(1);
+    Assert.assertEquals(0, protobufMessage.getEcReplicationConfig().getData());
+
+
+    //when EC config is NOT empty
+    subject = MockPipeline.createEcPipeline();
+
+    protobufMessage = subject.getProtobufMessage(1);
+    Assert.assertEquals(3, protobufMessage.getEcReplicationConfig().getData());
+    Assert
+        .assertEquals(2, protobufMessage.getEcReplicationConfig().getParity());
+
+  }
+
+  @Test
+  public void testReplicaIndexesSerialisedCorrectly() throws IOException {
+    Pipeline pipeline = MockPipeline.createEcPipeline();
+    HddsProtos.Pipeline protobufMessage = pipeline.getProtobufMessage(1);
+    Pipeline reloadedPipeline = Pipeline.getFromProtobuf(protobufMessage);
+
+    for (DatanodeDetails dn : pipeline.getNodes()) {
+      Assert.assertEquals(pipeline.getReplicaIndex(dn),
+          reloadedPipeline.getReplicaIndex(dn));
+    }
+  }
+
+  @Test
+  public void testECPipelineIsAlwaysHealthy() throws IOException {
+    Pipeline pipeline = MockPipeline.createEcPipeline();
+    Assert.assertTrue(pipeline.isHealthy());
+  }
+
+  @Test
+  public void testBuilderCopiesAllFieldsFromOtherPipeline() {
+    Pipeline original = MockPipeline.createEcPipeline();
+    Pipeline copied = Pipeline.newBuilder(original).build();
+    Assert.assertEquals(original.getId(), copied.getId());
+    Assert.assertEquals(original.getReplicationConfig(),
+        copied.getReplicationConfig());
+    Assert.assertEquals(original.getPipelineState(), copied.getPipelineState());
+    Assert.assertEquals(original.getId(), copied.getId());
+    Assert.assertEquals(original.getId(), copied.getId());
+    Assert.assertEquals(original.getId(), copied.getId());
+    Assert.assertEquals(original.getNodeSet(), copied.getNodeSet());
+    Assert.assertEquals(original.getNodesInOrder(), copied.getNodesInOrder());
+    Assert.assertEquals(original.getLeaderId(), copied.getLeaderId());
+    Assert.assertEquals(original.getCreationTimestamp(),
+        copied.getCreationTimestamp());
+    Assert.assertEquals(original.getSuggestedLeaderId(),
+        copied.getSuggestedLeaderId());
+    for (DatanodeDetails dn : original.getNodes()) {
+      Assert.assertEquals(original.getReplicaIndex(dn),
+          copied.getReplicaIndex(dn));
+    }
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
index 2b63183..a8b83a1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -46,6 +46,7 @@
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
@@ -193,7 +194,9 @@
       String storedChecksum = containerData.getChecksum();
 
       Yaml yaml = ContainerDataYaml.getYamlForContainerType(
-              containerData.getContainerType());
+          containerData.getContainerType(),
+          containerData instanceof KeyValueContainerData &&
+              ((KeyValueContainerData)containerData).getReplicaIndex() > 0);
       containerData.computeAndSetChecksum(yaml);
       String computedChecksum = containerData.getChecksum();
 
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
index b4e15db..d3caa31 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -27,6 +27,7 @@
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -42,8 +43,11 @@
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 
 import com.google.common.base.Preconditions;
+
+import static org.apache.hadoop.ozone.OzoneConsts.REPLICA_INDEX;
 import static org.apache.hadoop.ozone.container.keyvalue
     .KeyValueContainerData.KEYVALUE_YAML_TAG;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
@@ -84,8 +88,12 @@
     Writer writer = null;
     FileOutputStream out = null;
     try {
+      boolean withReplicaIndex =
+          containerData instanceof KeyValueContainerData &&
+          ((KeyValueContainerData) containerData).getReplicaIndex() > 0;
+
       // Create Yaml for given container type
-      Yaml yaml = getYamlForContainerType(containerType);
+      Yaml yaml = getYamlForContainerType(containerType, withReplicaIndex);
       // Compute Checksum and update ContainerData
       containerData.computeAndSetChecksum(yaml);
 
@@ -148,7 +156,8 @@
     propertyUtils.setBeanAccess(BeanAccess.FIELD);
     propertyUtils.setAllowReadOnlyProperties(true);
 
-    Representer representer = new ContainerDataRepresenter();
+    Representer representer = new ContainerDataRepresenter(
+        KeyValueContainerData.getYamlFields());
     representer.setPropertyUtils(propertyUtils);
 
     Constructor containerDataConstructor = new ContainerDataConstructor();
@@ -166,39 +175,51 @@
    * Given a ContainerType this method returns a Yaml representation of
    * the container properties.
    *
-   * @param containerType type of container
+   * @param containerType    type of container
+   * @param withReplicaIndex in the container yaml
    * @return Yamal representation of container properties
-   *
    * @throws StorageContainerException if the type is unrecognized
    */
-  public static Yaml getYamlForContainerType(ContainerType containerType)
+  public static Yaml getYamlForContainerType(ContainerType containerType,
+      boolean withReplicaIndex)
       throws StorageContainerException {
     PropertyUtils propertyUtils = new PropertyUtils();
     propertyUtils.setBeanAccess(BeanAccess.FIELD);
     propertyUtils.setAllowReadOnlyProperties(true);
 
-    switch (containerType) {
-    case KeyValueContainer:
-      Representer representer = new ContainerDataRepresenter();
+    if (containerType == ContainerType.KeyValueContainer) {
+      List<String> yamlFields =
+          KeyValueContainerData.getYamlFields();
+      if (withReplicaIndex) {
+        yamlFields = new ArrayList<>(yamlFields);
+        yamlFields.add(REPLICA_INDEX);
+      }
+      Representer representer = new ContainerDataRepresenter(yamlFields);
       representer.setPropertyUtils(propertyUtils);
       representer.addClassTag(
           KeyValueContainerData.class,
-          KeyValueContainerData.KEYVALUE_YAML_TAG);
+          KEYVALUE_YAML_TAG);
 
       Constructor keyValueDataConstructor = new ContainerDataConstructor();
 
       return new Yaml(keyValueDataConstructor, representer);
-    default:
-      throw new StorageContainerException("Unrecognized container Type " +
-          "format " + containerType, ContainerProtos.Result
-          .UNKNOWN_CONTAINER_TYPE);
     }
+    throw new StorageContainerException("Unrecognized container Type " +
+        "format " + containerType, ContainerProtos.Result
+        .UNKNOWN_CONTAINER_TYPE);
   }
 
   /**
    * Representer class to define which fields need to be stored in yaml file.
    */
   private static class ContainerDataRepresenter extends Representer {
+
+    private List<String> yamlFields;
+
+    ContainerDataRepresenter(List<String> yamlFields) {
+      this.yamlFields = yamlFields;
+    }
+
     @Override
     protected Set<Property> getProperties(Class<? extends Object> type) {
       Set<Property> set = super.getProperties(type);
@@ -207,7 +228,7 @@
       // When a new Container type is added, we need to add what fields need
       // to be filtered here
       if (type.equals(KeyValueContainerData.class)) {
-        List<String> yamlFields = KeyValueContainerData.getYamlFields();
+
         // filter properties
         for (Property prop : set) {
           String name = prop.getName();
@@ -282,7 +303,11 @@
             .setState(ContainerProtos.ContainerDataProto.State.valueOf(state));
         String schemaVersion = (String) nodes.get(OzoneConsts.SCHEMA_VERSION);
         kvData.setSchemaVersion(schemaVersion);
-
+        final Object replicaIndex = nodes.get(REPLICA_INDEX);
+        if (replicaIndex != null) {
+          kvData.setReplicaIndex(
+              ((Long) replicaIndex).intValue());
+        }
         return kvData;
       }
     }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index 1edd046..83ba104 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -409,6 +409,11 @@
         ContainerProtos.ContainerType.KeyValueContainer;
     createRequest.setContainerType(containerType);
 
+    if (containerRequest.hasWriteChunk()) {
+      createRequest.setReplicaIndex(
+          containerRequest.getWriteChunk().getBlockID().getReplicaIndex());
+    }
+
     ContainerCommandRequestProto.Builder requestBuilder =
         ContainerCommandRequestProto.newBuilder()
             .setCmdType(Type.CreateContainer)
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index ba131ff..fdf44ed 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -754,6 +754,7 @@
         .setKeyCount(containerData.getBlockCount())
         .setUsed(containerData.getBytesUsed())
         .setState(getHddsState())
+        .setReplicaIndex(containerData.getReplicaIndex())
         .setDeleteTransactionId(containerData.getDeleteTransactionId())
         .setBlockCommitSequenceId(containerData.getBlockCommitSequenceId())
         .setOriginNodeId(containerData.getOriginNodeId());
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index ccc252a..0bc6c3a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -83,6 +83,8 @@
 
   private long blockCommitSequenceId;
 
+  private int replicaIndex;
+
   static {
     // Initialize YAML fields
     KV_YAML_FIELDS = Lists.newArrayList();
@@ -303,4 +305,12 @@
 
     db.getStore().getBatchHandler().commitBatchOperation(batchOperation);
   }
+
+  public int getReplicaIndex() {
+    return replicaIndex;
+  }
+
+  public void setReplicaIndex(int replicaIndex) {
+    this.replicaIndex = replicaIndex;
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index b2e56c4..cce6c3e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -273,6 +273,8 @@
     KeyValueContainerData newContainerData = new KeyValueContainerData(
         containerID, layoutVersion, maxContainerSize, request.getPipelineID(),
         getDatanodeId());
+    newContainerData.setReplicaIndex(request.getCreateContainer()
+        .getReplicaIndex());
     // TODO: Add support to add metadataList to ContainerData. Add metadata
     // to container during creation.
     KeyValueContainer newContainer = new KeyValueContainer(
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
index a80adca..7cee63d 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
@@ -571,7 +571,8 @@
     // Changing the paths above affects the checksum, so it was also removed
     // from the container file and calculated at run time.
     Yaml yaml = ContainerDataYaml.getYamlForContainerType(
-            kvData.getContainerType());
+            kvData.getContainerType(),
+        kvData.getReplicaIndex() > 0);
     kvData.computeAndSetChecksum(yaml);
 
     KeyValueContainerUtil.parseKVContainerData(kvData, conf);
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
index 0bfdb17..c0843ab 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.container.common.impl;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileUtil;
@@ -28,6 +29,7 @@
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.junit.Assert;
 import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
 import org.apache.ozone.test.GenericTestUtils;
 import org.junit.Test;
@@ -36,6 +38,7 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.time.Instant;
 import java.util.UUID;
 
@@ -77,7 +80,8 @@
    * Creates a .container file. cleanup() should be called at the end of the
    * test when container file is created.
    */
-  private File createContainerFile(long containerID) throws IOException {
+  private File createContainerFile(long containerID, int replicaIndex)
+      throws IOException {
     new File(testRoot).mkdirs();
 
     String containerPath = containerID + ".container";
@@ -92,6 +96,7 @@
     keyValueContainerData.updateDataScanTime(SCAN_TIME);
     keyValueContainerData.setSchemaVersion(
         VersionedDatanodeFeatures.SchemaV2.chooseSchemaVersion());
+    keyValueContainerData.setReplicaIndex(replicaIndex);
 
     File containerFile = new File(testRoot, containerPath);
 
@@ -113,7 +118,7 @@
   public void testCreateContainerFile() throws IOException {
     long containerID = testContainerID++;
 
-    File containerFile = createContainerFile(containerID);
+    File containerFile = createContainerFile(containerID, 7);
 
     // Read from .container file, and verify data.
     KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
@@ -136,7 +141,8 @@
     assertEquals(SCAN_TIME.toEpochMilli(),
                  kvData.getDataScanTimestamp().longValue());
     assertEquals(VersionedDatanodeFeatures.SchemaV2.chooseSchemaVersion(),
-            kvData.getSchemaVersion());
+        kvData.getSchemaVersion());
+    assertEquals(7, kvData.getReplicaIndex());
 
     // Update ContainerData.
     kvData.addMetadata(OzoneConsts.VOLUME, VOLUME_OWNER);
@@ -174,6 +180,21 @@
   }
 
   @Test
+  public void testCreateContainerFileWithoutReplicaIndex() throws IOException {
+    long containerID = testContainerID++;
+
+    File containerFile = createContainerFile(containerID, 0);
+
+    final String content =
+        FileUtils.readFileToString(containerFile, Charset.defaultCharset());
+
+    Assert.assertFalse("ReplicaIndex shouldn't be persisted if zero",
+        content.contains("replicaIndex"));
+    cleanup();
+  }
+
+
+  @Test
   public void testIncorrectContainerFile() throws IOException {
     try {
       String containerFile = "incorrect.container";
@@ -233,7 +254,24 @@
   public void testChecksumInContainerFile() throws IOException {
     long containerID = testContainerID++;
 
-    File containerFile = createContainerFile(containerID);
+    File containerFile = createContainerFile(containerID, 0);
+
+    // Read from .container file, and verify data.
+    KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
+        .readContainerFile(containerFile);
+    ContainerUtils.verifyChecksum(kvData, conf);
+
+    cleanup();
+  }
+
+  /**
+   * Test to verify {@link ContainerUtils#verifyChecksum(ContainerData)}.
+   */
+  @Test
+  public void testChecksumInContainerFileWithReplicaIndex() throws IOException {
+    long containerID = testContainerID++;
+
+    File containerFile = createContainerFile(containerID, 10);
 
     // Read from .container file, and verify data.
     KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java
index d882ca4..34fb79a 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java
@@ -192,7 +192,7 @@
     // restarted with SCM HA config and gets a different SCM ID.
     conf.setBoolean(ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY, true);
     changeScmID();
-    restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion());
+    restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true);
     // Make sure the existing container can be read.
     readChunk(exportWriteChunk2, pipeline);
 
@@ -289,7 +289,7 @@
 
     /// FINALIZED: Restart datanode to upgrade the failed volume ///
 
-    restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion());
+    restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion(), false);
 
     Assert.assertEquals(1,
         dsm.getContainer().getVolumeSet().getVolumesList().size());
@@ -344,7 +344,7 @@
     changeScmID();
     // A new volume is added that must be formatted.
     File preFinVolume2 = addVolume();
-    restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion());
+    restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true);
 
     Assert.assertEquals(2,
         dsm.getContainer().getVolumeSet().getVolumesList().size());
@@ -378,7 +378,7 @@
     File finVolume = addVolume();
     // Yet another SCM ID is received this time, but it should not matter.
     changeScmID();
-    restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion());
+    restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion(), false);
     Assert.assertEquals(3,
         dsm.getContainer().getVolumeSet().getVolumesList().size());
     Assert.assertEquals(0,
@@ -521,7 +521,7 @@
     callVersionEndpointTask();
   }
 
-  public void restartDatanode(int expectedMlv)
+  public void restartDatanode(int expectedMlv, boolean exactMatch)
       throws Exception {
     // Stop existing datanode.
     DatanodeDetails dd = dsm.getDatanodeDetails();
@@ -532,7 +532,12 @@
         conf, null, null,
         null);
     int mlv = dsm.getLayoutVersionManager().getMetadataLayoutVersion();
-    Assert.assertEquals(expectedMlv, mlv);
+    if (exactMatch) {
+      Assert.assertEquals(expectedMlv, mlv);
+    } else {
+      Assert.assertTrue("Expected minimum mlv(" + expectedMlv
+          + ") is smaller than mlv(" + mlv + ").", expectedMlv <= mlv);
+    }
 
     callVersionEndpointTask();
   }
diff --git a/hadoop-hdds/docs/content/feature/EC-Chunk-Layout.png b/hadoop-hdds/docs/content/feature/EC-Chunk-Layout.png
new file mode 100644
index 0000000..d7b6785
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/EC-Chunk-Layout.png
Binary files differ
diff --git a/hadoop-hdds/docs/content/feature/EC-Reads-With-No-Failures.png b/hadoop-hdds/docs/content/feature/EC-Reads-With-No-Failures.png
new file mode 100644
index 0000000..7dd4cab
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/EC-Reads-With-No-Failures.png
Binary files differ
diff --git a/hadoop-hdds/docs/content/feature/EC-Reconstructional-Read.png b/hadoop-hdds/docs/content/feature/EC-Reconstructional-Read.png
new file mode 100644
index 0000000..427e73a
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/EC-Reconstructional-Read.png
Binary files differ
diff --git a/hadoop-hdds/docs/content/feature/EC-Write-Block-Allocation-in-Containers.png b/hadoop-hdds/docs/content/feature/EC-Write-Block-Allocation-in-Containers.png
new file mode 100644
index 0000000..3945662
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/EC-Write-Block-Allocation-in-Containers.png
Binary files differ
diff --git a/hadoop-hdds/docs/content/feature/ErasureCoding.md b/hadoop-hdds/docs/content/feature/ErasureCoding.md
new file mode 100644
index 0000000..cced272
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/ErasureCoding.md
@@ -0,0 +1,212 @@
+---
+title: "Ozone Erasure Coding"
+weight: 1
+menu:
+   main:
+      parent: Features
+summary: Erasure Coding Support for Ozone.
+---
+<!---
+  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.
+-->
+
+## Background
+
+Distributed systems basic expectation is to provide the data durability.
+To provide the higher data durability, many popular storage systems use replication
+approach which is expensive. The Apache Ozone supports `RATIS/THREE` replication scheme.
+The Ozone default replication scheme `RATIS/THREE` has 200% overhead in storage
+space and other resources (e.g., network bandwidth).
+However, for warm and cold datasets with relatively low I/O activities, additional
+block replicas rarely accessed during normal operations, but still consume the same
+amount of resources as the first replica.
+
+Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication,
+which provides the same level of fault-tolerance with much less storage space.
+In typical EC setups, the storage overhead is no more than 50%. The replication factor of an EC file is meaningless.
+Instead of replication factor, we introduced ReplicationConfig interface to specify the required type of replication,
+either `RATIS/THREE` or `EC`.
+
+Integrating EC with Ozone can improve storage efficiency while still providing similar
+data durability as traditional replication-based Ozone deployments.
+As an example, a 3x replicated file with 6 blocks will consume 6*3 = `18` blocks of disk space.
+But with EC (6 data, 3 parity) deployment, it will only consume `9` blocks of disk space.
+
+## Architecture
+
+The storage data layout is a key factor in the implementation of EC. After deep analysis
+and several technical consideration, the most fitting data layout is striping model.
+The data striping layout is not new. The striping model already adapted by several other
+file systems(Ex: Quantcast File System, Hadoop Distributed File System etc) successfully before.
+
+For example, with the EC (6 data, 3 parity) scheme, the data chunks will be distributed to first 6 data nodes in order
+and then client generates the 3 parity chunks and transfer to remaining 3 nodes in order.
+These 9 chunks together we call as "Stripe". Next 6 chunks will be distributed to the same first 6 data nodes again
+and the parity to remaining 3 nodes. These 9 data nodes stored blocks together called as "BlockGroup".
+
+If the application is continuing to write beyond the size of `6 * BLOCK_SIZE`, then client will request new block group from Ozone Manager.
+
+### Erasure Coding Write
+
+The core logic of erasure coding writes are placed at ozone client.
+When client creates the file, ozone manager allocates the block group(`d + p`)
+number of nodes from the pipeline provider and return the same to client.
+As data is coming in from the application, client will write first d number of chunks
+to d number of data nodes in block group. It will also cache the d number chunks
+to generate the parity chunks. Once parity chunks generated, it will transfer the
+same to the remaining p nodes in order. Once all blocks reached their configured sizes,
+client will request the new block group nodes.
+
+Below diagram depicts the block allocation in containers as logical groups.
+For interest of space, we assumed EC(3, 2) Replication Config for the diagram.
+
+![EC Block Allocation in Containers](EC-Write-Block-Allocation-in-Containers.png)
+
+
+Let's zoom out the blockID: 1 data layout from the above picture, that showed in the following picture.
+This picture shows how the chunks will be layed out in data node blocks.
+
+![EC Chunk Layout](EC-Chunk-Layout.png)
+
+Currently, the EC client re-used the data transfer end-points to transfer the data to data nodes.
+The XceiverClientGRPC client used for writing data and putBlock info.
+The datanode side changes are minimal as we reused the same existing transfer protocols.
+The EC data block written at the datanode is same as any other block in non-EC mode.
+In a single block group, container id numbers are same in all nodes. A file can have multiple block groups.
+Each block group will have `d+p` number of block and all ids are same.
+
+**d** - Number of data blocks in a block group
+
+**p** - Number of parity blocks in a block group
+
+### Erasure Coding Read
+
+For reads, OM will provide the node location details as part of key lookup.
+If the key is erasure coded, Ozone client reads it in EC fashion. Since the data layout
+is different(see the previous section about write path), reads should consider the layout and do the reads accordingly.
+
+The EC client will open the connections to DNs based on the expected locations. When all data locations are available,
+it will attempt to do plain reads chunk by chunk in round robin fashion from d data blocks.
+
+Below picture shows the order when there are no failures while reading.
+
+![EC Reads With no Failures](EC-Reads-With-No-Failures.png)
+
+Until it sees read failures, there is no need of doing EC reconstruction.
+
+#### Erasure Coding On-the-fly Reconstruction Reads
+
+When client detects there are failures while reading or when starting the reads,
+Ozone EC client is capable of reconstructing/recovering the lost data by doing the EC decoding.
+To do the EC decoding it needs to read parity replicas. This is a degraded read as it needs to do reconstruction.
+This reconstruction is completely transparent to the applications.
+
+Below picture depicts how it uses parity replicas in reconstruction.
+
+![EC Reconstructional Reads](EC-Reconstructional-Read.png)
+
+ ### Erasure Coding Replication Config
+
+ Apache Ozone built with the pure 'Object Storage' semantics. However, many big data
+ eco system projects still uses file system APIs. To provide both worlds best access to Ozone,
+ it's provided both faces of interfaces. In both cases, keys/files would be written into buckets under the hood.
+ So, EC Replication Configs can be set at bucket level.
+ The EC policy encapsulates how to encode/decode a file.
+ Each EC Replication Config defined by the following pieces of information:
+  1. **data:** Data blocks number in an EC block group.
+  2. **parity:** Parity blocks number in an EC block group.
+  3. **ecChunkSize:** The size of a striping chunk. This determines the granularity of striped reads and writes.
+  4. **codec:** This is to indicate the type of EC algorithms (e.g., `RS`(Reed-Solomon), `XOR`).
+
+To pass the EC Replication Config in command line or configuration files, we need to use the following format:
+*codec*-*num data blocks*-*num parity blocks*-*ec chunk size*
+
+Currently, there are three built-in EC Replication Configs supported: `RS-3-2-1024k`, `RS-6-3-1024k`, `XOR-2-1-1024k`.
+The most recommended option is `RS-6-3-1024k`. When a key/file created without specifying the Replication Config,
+it inherits the EC Replication Config of its bucket if it's available.
+
+Changing the bucket level EC Replication Config only affect new files created within the bucket.
+Once a file has been created, its EC Replication Config cannot be changed currently.
+
+Deployment
+----------
+### Cluster and Hardware Configuration
+
+EC places additional demands on the cluster in terms of CPU and network.
+Encoding and decoding work consumes additional CPU on both Ozone clients and DataNodes.
+EC requires a minimum of as many DataNodes in the cluster as the configured EC stripe width. For the EC Replication Config `RS` (6,3), we need
+a minimum of 9 DataNodes.
+
+Erasure Coded keys/files also spread across racks for rack fault-tolerance.
+This means that when reading and writing striped files, most operations are off-rack.
+Network bisection bandwidth is thus very important.
+
+For rack fault-tolerance, it is also important to have enough number of racks,
+so that on average, each rack holds number of blocks no more than the number of EC parity blocks.
+A formula to calculate this would be (data blocks + parity blocks) / parity blocks, rounding up.
+For `RS` (6,3) EC Replication Config, this means minimally 3 racks (calculated by (6 + 3) / 3 = 3),
+and ideally 9 or more to handle planned and unplanned outages.
+For clusters with fewer racks than the number of the parity cells, Ozone cannot maintain rack fault-tolerance,
+but will still attempt to spread a striped file across multiple nodes to preserve node-level fault-tolerance.
+Due to this reason, it is recommended to setup racks with similar number of DataNodes.
+
+### Configurations
+
+EC Replication Config can be enabled at bucket level as discussed above.
+Cluster wide default Replication Config can be set with EC Replication Config by using
+the configuration keys `ozone.server.default.replication.type` and `ozone.server.default.replication`.
+
+```XML
+<property>
+   <name>ozone.server.default.replication.type</name>
+   <value>EC</value>
+</property>
+
+<property>
+   <name>ozone.server.default.replication</name>
+   <value>RS-6-3-1024k</value>
+</property>
+```
+
+Please note, the above configurations will be used only when client does not pass
+any replication config or bucket does not have any default values.
+
+#### Setting EC Replication Config On Bucket
+
+We can set the bucket EC Replication Config via ozone sh command. The EC Replication Config options can be passed while creating the bucket.
+
+```shell
+ozone sh bucket create <bucket path> --type EC --replication rs-6-3-1024k
+```
+
+We can also reset the EC Replication Config with the following command.
+
+```shell
+ozone sh bucket set-replication-config <bucket path> --type EC --replication rs-3-2-1024k
+```
+
+Once we reset, only newly created keys take effect of this new setting. Prior created keys in the bucket stay with same older setting.
+
+#### Setting EC Replication Config While Creating Keys/Files
+
+We can pass the EC Replication Config while creating the keys irrespective of bucket Replication Config.
+
+```shell
+ozone sh key put <Ozone Key Object Path> <Local File> --type EC --replication rs-6-3-1024k
+```
+
+In the case bucket already has default EC Replication Config, there is no need of passing EC Replication Config while creating key.
+
diff --git a/hadoop-hdds/erasurecode/pom.xml b/hadoop-hdds/erasurecode/pom.xml
new file mode 100644
index 0000000..ca2e3cb
--- /dev/null
+++ b/hadoop-hdds/erasurecode/pom.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+https://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.ozone</groupId>
+    <artifactId>hdds</artifactId>
+    <version>1.3.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>hdds-erasurecode</artifactId>
+  <version>1.3.0-SNAPSHOT</version>
+  <description>Apache Ozone Distributed Data Store Earsurecode utils
+  </description>
+  <name>Apache Ozone HDDS Erasurecode</name>
+  <packaging>jar</packaging>
+
+  <dependencies>
+
+    <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-test-utils</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.github.spotbugs</groupId>
+      <artifactId>spotbugs-annotations</artifactId>
+      <scope>compile</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+
+  </build>
+</project>
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/HadoopNativeECAccessorUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/HadoopNativeECAccessorUtil.java
new file mode 100644
index 0000000..5314db3
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/HadoopNativeECAccessorUtil.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+
+/**
+ * This class is used to access some of the protected API
+ * from hadoop native EC java code.
+ * This is needed until we adapt the native ec code from hadoop.
+ */
+@InterfaceAudience.Private
+public final class HadoopNativeECAccessorUtil {
+  public static void performEncodeImpl(NativeRSRawEncoder encoder,
+      ByteBuffer[] inputs, int[] inputOffsets, int dataLen,
+      ByteBuffer[] outputs, int[] outputOffsets) throws IOException {
+    encoder.performEncodeImpl(inputs, inputOffsets, dataLen, outputs,
+        outputOffsets);
+  }
+
+  public static void performDecodeImpl(NativeRSRawDecoder decoder,
+      ByteBuffer[] inputs, int[] inputOffsets, int dataLen, int[] erased,
+      ByteBuffer[] outputs, int[] outputOffsets) throws IOException {
+    decoder.performDecodeImpl(inputs, inputOffsets, dataLen, erased, outputs,
+        outputOffsets);
+  }
+
+  public static void performEncodeImpl(NativeXORRawEncoder encoder,
+      ByteBuffer[] inputs, int[] inputOffsets, int dataLen,
+      ByteBuffer[] outputs, int[] outputOffsets) throws IOException {
+    encoder.performEncodeImpl(inputs, inputOffsets, dataLen, outputs,
+        outputOffsets);
+  }
+
+  public static void performDecodeImpl(NativeXORRawDecoder decoder,
+      ByteBuffer[] inputs, int[] inputOffsets, int dataLen, int[] erased,
+      ByteBuffer[] outputs, int[] outputOffsets) throws IOException {
+    decoder.performDecodeImpl(inputs, inputOffsets, dataLen, erased, outputs,
+        outputOffsets);
+  }
+
+  private HadoopNativeECAccessorUtil() {
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/package-info.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/package-info.java
new file mode 100644
index 0000000..42c6705
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/package-info.java
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+/**
+ *
+ * Erasure coding rawcoder utilities for Apache Ozone.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/BufferAllocator.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/BufferAllocator.java
new file mode 100644
index 0000000..ed2c94e
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/BufferAllocator.java
@@ -0,0 +1,91 @@
+/**
+ * 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.ozone.erasurecode;
+
+
+import java.nio.ByteBuffer;
+
+/**
+ * An abstract buffer allocator used for test.
+ */
+public abstract class BufferAllocator {
+  private boolean usingDirect = false;
+
+  public BufferAllocator(boolean usingDirect) {
+    this.usingDirect = usingDirect;
+  }
+
+  protected boolean isUsingDirect() {
+    return usingDirect;
+  }
+
+  /**
+   * Allocate and return a ByteBuffer of specified length.
+   * @param bufferLen
+   * @return
+   */
+  public abstract ByteBuffer allocate(int bufferLen);
+
+  /**
+   * A simple buffer allocator that just uses ByteBuffer's
+   * allocate/allocateDirect API.
+   */
+  public static class SimpleBufferAllocator extends BufferAllocator {
+
+    public SimpleBufferAllocator(boolean usingDirect) {
+      super(usingDirect);
+    }
+
+    @Override
+    public ByteBuffer allocate(int bufferLen) {
+      return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) :
+          ByteBuffer.allocate(bufferLen);
+    }
+  }
+
+  /**
+   * A buffer allocator that allocates a buffer from an existing large buffer by
+   * slice calling, but if no available space just degrades as
+   * SimpleBufferAllocator. So please ensure enough space for it.
+   */
+  public static class SlicedBufferAllocator extends BufferAllocator {
+    private ByteBuffer overallBuffer;
+
+    public SlicedBufferAllocator(boolean usingDirect, int totalBufferLen) {
+      super(usingDirect);
+      overallBuffer = isUsingDirect() ?
+          ByteBuffer.allocateDirect(totalBufferLen) :
+          ByteBuffer.allocate(totalBufferLen);
+    }
+
+    @Override
+    public ByteBuffer allocate(int bufferLen) {
+      if (bufferLen > overallBuffer.capacity() - overallBuffer.position()) {
+        // If no available space for the requested length, then allocate new
+        return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) :
+            ByteBuffer.allocate(bufferLen);
+      }
+
+      overallBuffer.limit(overallBuffer.position() + bufferLen);
+      ByteBuffer result = overallBuffer.slice();
+      overallBuffer.position(overallBuffer.position() + bufferLen);
+      return result;
+    }
+  }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/CodecRegistry.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/CodecRegistry.java
new file mode 100644
index 0000000..1c447a7
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/CodecRegistry.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.NativeXORRawErasureCoderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * This class registers all coder implementations.
+ *
+ * {@link CodecRegistry} maps codec names to coder factories. All coder
+ * factories are dynamically identified and loaded using ServiceLoader.
+ */
+@InterfaceAudience.Private
+public final class CodecRegistry {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CodecRegistry.class);
+
+  private static CodecRegistry instance = new CodecRegistry();
+  private Map<String, List<RawErasureCoderFactory>> coderMap;
+  private Map<String, String[]> coderNameMap;
+
+  private CodecRegistry() {
+    coderMap = new HashMap<>();
+    coderNameMap = new HashMap<>();
+    final ServiceLoader<RawErasureCoderFactory> coderFactories =
+        ServiceLoader.load(RawErasureCoderFactory.class);
+    updateCoders(coderFactories);
+  }
+
+  public static CodecRegistry getInstance() {
+    return instance;
+  }
+
+  /**
+   * Update coderMap and coderNameMap with iterable type of coder factories.
+   * @param coderFactories
+   */
+  @VisibleForTesting
+  void updateCoders(Iterable<RawErasureCoderFactory> coderFactories) {
+    for (RawErasureCoderFactory coderFactory : coderFactories) {
+      String codecName = coderFactory.getCodecName();
+      List<RawErasureCoderFactory> coders = coderMap.get(codecName);
+      if (coders == null) {
+        coders = new ArrayList<>();
+        coders.add(coderFactory);
+        coderMap.put(codecName, coders);
+        LOG.debug("Codec registered: codec = {}, coder = {}",
+            coderFactory.getCodecName(), coderFactory.getCoderName());
+      } else {
+        Boolean hasConflit = false;
+        for (RawErasureCoderFactory coder : coders) {
+          if (coder.getCoderName().equals(coderFactory.getCoderName())) {
+            hasConflit = true;
+            LOG.error("Coder {} cannot be registered because its coder name " +
+                    "{} has conflict with {}",
+                coderFactory.getClass().getName(),
+                coderFactory.getCoderName(), coder.getClass().getName());
+            break;
+          }
+        }
+        if (!hasConflit) {
+          if (coderFactory instanceof NativeRSRawErasureCoderFactory
+              || coderFactory instanceof NativeXORRawErasureCoderFactory) {
+            coders.add(0, coderFactory);
+          } else {
+            coders.add(coderFactory);
+          }
+          LOG.debug("Codec registered: codec = {}, coder = {}",
+              coderFactory.getCodecName(), coderFactory.getCoderName());
+        }
+      }
+    }
+
+    // update coderNameMap accordingly
+    coderNameMap.clear();
+    for (Map.Entry<String, List<RawErasureCoderFactory>> entry :
+        coderMap.entrySet()) {
+      String codecName = entry.getKey();
+      List<RawErasureCoderFactory> coders = entry.getValue();
+      coderNameMap.put(codecName, coders.stream().
+          map(RawErasureCoderFactory::getCoderName).
+          collect(Collectors.toList()).toArray(new String[0]));
+    }
+  }
+
+  /**
+   * Get all coder names of the given codec.
+   * @param codecName the name of codec
+   * @return an array of all coder names, null if not exist
+   */
+  public String[] getCoderNames(String codecName) {
+    String[] coderNames = coderNameMap.get(codecName);
+    return coderNames;
+  }
+
+  /**
+   * Get all coder factories of the given codec.
+   * @param codecName the name of codec
+   * @return a list of all coder factories, null if not exist
+   */
+  public List<RawErasureCoderFactory> getCoders(String codecName) {
+    List<RawErasureCoderFactory> coders = coderMap.get(codecName);
+    return coders;
+  }
+
+  /**
+   * Get all codec names.
+   * @return a set of all codec names
+   */
+  public Set<String> getCodecNames() {
+    return coderMap.keySet();
+  }
+
+  /**
+   * Get a specific coder factory defined by codec name and coder name.
+   * @param codecName name of the codec
+   * @param coderName name of the coder
+   * @return the specific coder, null if not exist
+   */
+  public RawErasureCoderFactory getCoderByName(
+      String codecName, String coderName) {
+    List<RawErasureCoderFactory> coders = getCoders(codecName);
+
+    // find the RawErasureCoderFactory with the name of coderName
+    for (RawErasureCoderFactory coder : coders) {
+      if (coder.getCoderName().equals(coderName)) {
+        return coder;
+      }
+    }
+    return null;
+  }
+
+  public RawErasureCoderFactory getCodecFactory(String codecName) {
+    for (RawErasureCoderFactory factory : getCoders(codecName)) {
+      return factory;
+    }
+    throw new IllegalArgumentException("There is no registered codec " +
+        "factory for codec " + codecName);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/ECChunk.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/ECChunk.java
new file mode 100644
index 0000000..1bf28b3
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/ECChunk.java
@@ -0,0 +1,113 @@
+/**
+ * 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.ozone.erasurecode;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A wrapper for ByteBuffer or bytes array for an erasure code chunk.
+ */
+public class ECChunk {
+
+  private ByteBuffer chunkBuffer;
+
+  // TODO: should be in a more general flags
+  private boolean allZero = false;
+
+  /**
+   * Wrapping a ByteBuffer.
+   *
+   * @param buffer buffer to be wrapped by the chunk
+   */
+  public ECChunk(ByteBuffer buffer) {
+    this.chunkBuffer = buffer;
+  }
+
+  public ECChunk(ByteBuffer buffer, int offset, int len) {
+    ByteBuffer tmp = buffer.duplicate();
+    tmp.position(offset);
+    tmp.limit(offset + len);
+    this.chunkBuffer = tmp.slice();
+  }
+
+  /**
+   * Wrapping a bytes array.
+   *
+   * @param buffer buffer to be wrapped by the chunk
+   */
+  public ECChunk(byte[] buffer) {
+    this.chunkBuffer = ByteBuffer.wrap(buffer);
+  }
+
+  public ECChunk(byte[] buffer, int offset, int len) {
+    this.chunkBuffer = ByteBuffer.wrap(buffer, offset, len);
+  }
+
+  public boolean isAllZero() {
+    return allZero;
+  }
+
+  public void setAllZero(boolean allZero) {
+    this.allZero = allZero;
+  }
+
+  /**
+   * Convert to ByteBuffer.
+   *
+   * @return ByteBuffer
+   */
+  public ByteBuffer getBuffer() {
+    return chunkBuffer;
+  }
+
+  /**
+   * Convert an array of this chunks to an array of ByteBuffers.
+   *
+   * @param chunks chunks to convert into buffers
+   * @return an array of ByteBuffers
+   */
+  public static ByteBuffer[] toBuffers(ECChunk[] chunks) {
+    ByteBuffer[] buffers = new ByteBuffer[chunks.length];
+
+    ECChunk chunk;
+    for (int i = 0; i < chunks.length; i++) {
+      chunk = chunks[i];
+      if (chunk == null) {
+        buffers[i] = null;
+      } else {
+        buffers[i] = chunk.getBuffer();
+      }
+    }
+
+    return buffers;
+  }
+
+  /**
+   * Convert to a bytes array, just for test usage.
+   * @return bytes array
+   */
+  public byte[] toBytesArray() {
+    byte[] bytesArr = new byte[chunkBuffer.remaining()];
+    // Avoid affecting the original one
+    chunkBuffer.mark();
+    chunkBuffer.get(bytesArr);
+    chunkBuffer.reset();
+
+    return bytesArr;
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/package-info.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/package-info.java
new file mode 100644
index 0000000..fe69252
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/package-info.java
@@ -0,0 +1,31 @@
+/**
+ * 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.
+ */
+
+/**
+ *
+ * Erasure coding utilities for Apache Ozone.
+ *
+ * Initial implementation is imported from Apache Hadoop see that repo for full
+ * history.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.ozone.erasurecode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawDecoder.java
new file mode 100644
index 0000000..537c2bb
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawDecoder.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.util.PerformanceAdvisory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * Abstract native raw decoder for all native coders to extend with.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+abstract class AbstractNativeRawDecoder extends RawErasureDecoder {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(AbstractNativeRawDecoder.class);
+
+  // Protect ISA-L coder data structure in native layer from being accessed and
+  // updated concurrently by the init, release and decode functions.
+  protected final ReentrantReadWriteLock decoderLock =
+      new ReentrantReadWriteLock();
+
+  AbstractNativeRawDecoder(ECReplicationConfig replicationConfig) {
+    super(replicationConfig);
+  }
+
+  @Override
+  protected void doDecode(ByteBufferDecodingState decodingState)
+      throws IOException {
+    decoderLock.readLock().lock();
+    try {
+      int[] inputOffsets = new int[decodingState.inputs.length];
+      int[] outputOffsets = new int[decodingState.outputs.length];
+
+      ByteBuffer buffer;
+      for (int i = 0; i < decodingState.inputs.length; ++i) {
+        buffer = decodingState.inputs[i];
+        if (buffer != null) {
+          inputOffsets[i] = buffer.position();
+        }
+      }
+
+      for (int i = 0; i < decodingState.outputs.length; ++i) {
+        buffer = decodingState.outputs[i];
+        outputOffsets[i] = buffer.position();
+      }
+
+      performDecodeImpl(decodingState.inputs, inputOffsets,
+          decodingState.decodeLength, decodingState.erasedIndexes,
+          decodingState.outputs, outputOffsets);
+    } finally {
+      decoderLock.readLock().unlock();
+    }
+  }
+
+  protected abstract void performDecodeImpl(ByteBuffer[] inputs,
+      int[] inputOffsets, int dataLen,
+      int[] erased, ByteBuffer[] outputs,
+      int[] outputOffsets)
+      throws IOException;
+
+  @Override
+  protected void doDecode(ByteArrayDecodingState decodingState)
+      throws IOException {
+    PerformanceAdvisory.LOG.debug("convertToByteBufferState is invoked, " +
+        "not efficiently. Please use direct ByteBuffer inputs/outputs");
+
+    ByteBufferDecodingState bbdState = decodingState.convertToByteBufferState();
+    doDecode(bbdState);
+
+    for (int i = 0; i < decodingState.outputs.length; i++) {
+      bbdState.outputs[i].get(decodingState.outputs[i],
+          decodingState.outputOffsets[i], decodingState.decodeLength);
+    }
+  }
+
+  @Override
+  public boolean preferDirectBuffer() {
+    return true;
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawEncoder.java
new file mode 100644
index 0000000..df6f36f
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawEncoder.java
@@ -0,0 +1,99 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.util.PerformanceAdvisory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * Abstract native raw encoder for all native coders to extend with.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+abstract class AbstractNativeRawEncoder extends RawErasureEncoder {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(AbstractNativeRawEncoder.class);
+
+  // Protect ISA-L coder data structure in native layer from being accessed and
+  // updated concurrently by the init, release and encode functions.
+  protected final ReentrantReadWriteLock encoderLock =
+      new ReentrantReadWriteLock();
+
+  AbstractNativeRawEncoder(ECReplicationConfig replicationConfig) {
+    super(replicationConfig);
+  }
+
+  @Override
+  protected void doEncode(ByteBufferEncodingState encodingState)
+      throws IOException {
+    encoderLock.readLock().lock();
+    try {
+      int[] inputOffsets = new int[encodingState.inputs.length];
+      int[] outputOffsets = new int[encodingState.outputs.length];
+      int dataLen = encodingState.inputs[0].remaining();
+
+      ByteBuffer buffer;
+      for (int i = 0; i < encodingState.inputs.length; ++i) {
+        buffer = encodingState.inputs[i];
+        inputOffsets[i] = buffer.position();
+      }
+
+      for (int i = 0; i < encodingState.outputs.length; ++i) {
+        buffer = encodingState.outputs[i];
+        outputOffsets[i] = buffer.position();
+      }
+
+      performEncodeImpl(encodingState.inputs, inputOffsets, dataLen,
+          encodingState.outputs, outputOffsets);
+    } finally {
+      encoderLock.readLock().unlock();
+    }
+  }
+
+  protected abstract void performEncodeImpl(
+          ByteBuffer[] inputs, int[] inputOffsets,
+          int dataLen, ByteBuffer[] outputs, int[] outputOffsets)
+      throws IOException;
+
+  @Override
+  protected void doEncode(ByteArrayEncodingState encodingState)
+      throws IOException {
+    PerformanceAdvisory.LOG.debug("convertToByteBufferState is invoked, " +
+        "not efficiently. Please use direct ByteBuffer inputs/outputs");
+
+    ByteBufferEncodingState bbeState = encodingState.convertToByteBufferState();
+    doEncode(bbeState);
+
+    for (int i = 0; i < encodingState.outputs.length; i++) {
+      bbeState.outputs[i].get(encodingState.outputs[i],
+          encodingState.outputOffsets[i], encodingState.encodeLength);
+    }
+  }
+
+  @Override
+  public boolean preferDirectBuffer() {
+    return true;
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayDecodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayDecodingState.java
new file mode 100644
index 0000000..3053539
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayDecodingState.java
@@ -0,0 +1,132 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+
+import java.nio.ByteBuffer;
+
+/**
+ * A utility class that maintains decoding state during a decode call using
+ * byte array inputs.
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class ByteArrayDecodingState extends DecodingState {
+  byte[][] inputs;
+  int[] inputOffsets;
+  int[] erasedIndexes;
+  byte[][] outputs;
+  int[] outputOffsets;
+
+  ByteArrayDecodingState(RawErasureDecoder decoder, byte[][] inputs,
+                         int[] erasedIndexes, byte[][] outputs) {
+    this.decoder = decoder;
+    this.inputs = inputs;
+    this.outputs = outputs;
+    this.erasedIndexes = erasedIndexes;
+    byte[] validInput = CoderUtil.findFirstValidInput(inputs);
+    this.decodeLength = validInput.length;
+
+    checkParameters(inputs, erasedIndexes, outputs);
+    checkInputBuffers(inputs);
+    checkOutputBuffers(outputs);
+
+    this.inputOffsets = new int[inputs.length]; // ALL ZERO
+    this.outputOffsets = new int[outputs.length]; // ALL ZERO
+  }
+
+  ByteArrayDecodingState(RawErasureDecoder decoder,
+                         int decodeLength,
+                         int[] erasedIndexes,
+                         byte[][] inputs,
+                         int[] inputOffsets,
+                         byte[][] outputs,
+                         int[] outputOffsets) {
+    this.decoder = decoder;
+    this.decodeLength = decodeLength;
+    this.erasedIndexes = erasedIndexes;
+    this.inputs = inputs;
+    this.outputs = outputs;
+    this.inputOffsets = inputOffsets;
+    this.outputOffsets = outputOffsets;
+  }
+
+  /**
+   * Convert to a ByteBufferDecodingState when it's backed by on-heap arrays.
+   */
+  ByteBufferDecodingState convertToByteBufferState() {
+    ByteBuffer[] newInputs = new ByteBuffer[inputs.length];
+    ByteBuffer[] newOutputs = new ByteBuffer[outputs.length];
+
+    for (int i = 0; i < inputs.length; i++) {
+      newInputs[i] = CoderUtil.cloneAsDirectByteBuffer(inputs[i],
+          inputOffsets[i], decodeLength);
+    }
+
+    for (int i = 0; i < outputs.length; i++) {
+      newOutputs[i] = ByteBuffer.allocateDirect(decodeLength);
+    }
+
+    ByteBufferDecodingState bbdState = new ByteBufferDecodingState(decoder,
+        decodeLength, erasedIndexes, newInputs, newOutputs);
+    return bbdState;
+  }
+
+  /**
+   * Check and ensure the buffers are of the desired length.
+   * @param buffers the buffers to check
+   */
+  void checkInputBuffers(byte[][] buffers) {
+    int validInputs = 0;
+
+    for (byte[] buffer : buffers) {
+      if (buffer == null) {
+        continue;
+      }
+
+      if (buffer.length != decodeLength) {
+        throw new IllegalArgumentException(
+            "Invalid buffer, not of length " + decodeLength);
+      }
+
+      validInputs++;
+    }
+
+    if (validInputs < decoder.getNumDataUnits()) {
+      throw new IllegalArgumentException(
+          "No enough valid inputs are provided, not recoverable");
+    }
+  }
+
+  /**
+   * Check and ensure the buffers are of the desired length.
+   * @param buffers the buffers to check
+   */
+  void checkOutputBuffers(byte[][] buffers) {
+    for (byte[] buffer : buffers) {
+      if (buffer == null) {
+        throw new IllegalArgumentException(
+            "Invalid buffer found, not allowing null");
+      }
+
+      if (buffer.length != decodeLength) {
+        throw new IllegalArgumentException(
+            "Invalid buffer not of length " + decodeLength);
+      }
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayEncodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayEncodingState.java
new file mode 100644
index 0000000..0e0def8
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayEncodingState.java
@@ -0,0 +1,105 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A utility class that maintains encoding state during an encode call using
+ * byte array inputs.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class ByteArrayEncodingState extends EncodingState {
+  byte[][] inputs;
+  byte[][] outputs;
+  int[] inputOffsets;
+  int[] outputOffsets;
+
+  ByteArrayEncodingState(RawErasureEncoder encoder,
+                         byte[][] inputs, byte[][] outputs) {
+    this.encoder = encoder;
+    byte[] validInput = CoderUtil.findFirstValidInput(inputs);
+    this.encodeLength = validInput.length;
+    this.inputs = inputs;
+    this.outputs = outputs;
+
+    checkParameters(inputs, outputs);
+    checkBuffers(inputs);
+    checkBuffers(outputs);
+
+    this.inputOffsets = new int[inputs.length]; // ALL ZERO
+    this.outputOffsets = new int[outputs.length]; // ALL ZERO
+  }
+
+  ByteArrayEncodingState(RawErasureEncoder encoder,
+                         int encodeLength,
+                         byte[][] inputs,
+                         int[] inputOffsets,
+                         byte[][] outputs,
+                         int[] outputOffsets) {
+    this.encoder = encoder;
+    this.encodeLength = encodeLength;
+    this.inputs = inputs;
+    this.outputs = outputs;
+    this.inputOffsets = inputOffsets;
+    this.outputOffsets = outputOffsets;
+  }
+
+  /**
+   * Convert to a ByteBufferEncodingState when it's backed by on-heap arrays.
+   */
+  ByteBufferEncodingState convertToByteBufferState() {
+    ByteBuffer[] newInputs = new ByteBuffer[inputs.length];
+    ByteBuffer[] newOutputs = new ByteBuffer[outputs.length];
+
+    for (int i = 0; i < inputs.length; i++) {
+      newInputs[i] = CoderUtil.cloneAsDirectByteBuffer(inputs[i],
+          inputOffsets[i], encodeLength);
+    }
+
+    for (int i = 0; i < outputs.length; i++) {
+      newOutputs[i] = ByteBuffer.allocateDirect(encodeLength);
+    }
+
+    ByteBufferEncodingState bbeState = new ByteBufferEncodingState(encoder,
+        encodeLength, newInputs, newOutputs);
+    return bbeState;
+  }
+
+  /**
+   * Check and ensure the buffers are of the desired length.
+   * @param buffers the buffers to check
+   */
+  void checkBuffers(byte[][] buffers) {
+    for (byte[] buffer : buffers) {
+      if (buffer == null) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer found, not allowing null");
+      }
+
+      if (buffer.length != encodeLength) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer not of length " + encodeLength);
+      }
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferDecodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferDecodingState.java
new file mode 100644
index 0000000..7b9ebf6
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferDecodingState.java
@@ -0,0 +1,143 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A utility class that maintains decoding state during a decode call using
+ * ByteBuffer inputs.
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class ByteBufferDecodingState extends DecodingState {
+  ByteBuffer[] inputs;
+  ByteBuffer[] outputs;
+  int[] erasedIndexes;
+  boolean usingDirectBuffer;
+
+  ByteBufferDecodingState(RawErasureDecoder decoder, ByteBuffer[] inputs,
+                          int[] erasedIndexes, ByteBuffer[] outputs) {
+    this.decoder = decoder;
+    this.inputs = inputs;
+    this.outputs = outputs;
+    this.erasedIndexes = erasedIndexes;
+    ByteBuffer validInput = CoderUtil.findFirstValidInput(inputs);
+    this.decodeLength = validInput.remaining();
+    this.usingDirectBuffer = validInput.isDirect();
+
+    checkParameters(inputs, erasedIndexes, outputs);
+    checkInputBuffers(inputs);
+    checkOutputBuffers(outputs);
+  }
+
+  ByteBufferDecodingState(RawErasureDecoder decoder,
+                         int decodeLength,
+                         int[] erasedIndexes,
+                         ByteBuffer[] inputs,
+                          ByteBuffer[] outputs) {
+    this.decoder = decoder;
+    this.decodeLength = decodeLength;
+    this.erasedIndexes = erasedIndexes;
+    this.inputs = inputs;
+    this.outputs = outputs;
+  }
+
+  /**
+   * Convert to a ByteArrayDecodingState when it's backed by on-heap arrays.
+   */
+  ByteArrayDecodingState convertToByteArrayState() {
+    int[] inputOffsets = new int[inputs.length];
+    int[] outputOffsets = new int[outputs.length];
+    byte[][] newInputs = new byte[inputs.length][];
+    byte[][] newOutputs = new byte[outputs.length][];
+
+    ByteBuffer buffer;
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      if (buffer != null) {
+        inputOffsets[i] = buffer.arrayOffset() + buffer.position();
+        newInputs[i] = buffer.array();
+      }
+    }
+
+    for (int i = 0; i < outputs.length; ++i) {
+      buffer = outputs[i];
+      outputOffsets[i] = buffer.arrayOffset() + buffer.position();
+      newOutputs[i] = buffer.array();
+    }
+
+    ByteArrayDecodingState baeState = new ByteArrayDecodingState(decoder,
+        decodeLength, erasedIndexes, newInputs,
+        inputOffsets, newOutputs, outputOffsets);
+    return baeState;
+  }
+
+  /**
+   * Check and ensure the buffers are of the desired length and type, direct
+   * buffers or not.
+   * @param buffers the buffers to check
+   */
+  void checkInputBuffers(ByteBuffer[] buffers) {
+    int validInputs = 0;
+
+    for (ByteBuffer buffer : buffers) {
+      if (buffer == null) {
+        continue;
+      }
+
+      if (buffer.remaining() != decodeLength) {
+        throw new IllegalArgumentException(
+            "Invalid buffer, not of length " + decodeLength);
+      }
+      if (buffer.isDirect() != usingDirectBuffer) {
+        throw new IllegalArgumentException(
+            "Invalid buffer, isDirect should be " + usingDirectBuffer);
+      }
+
+      validInputs++;
+    }
+
+    if (validInputs < decoder.getNumDataUnits()) {
+      throw new IllegalArgumentException(
+          "No enough valid inputs are provided, not recoverable");
+    }
+  }
+
+  /**
+   * Check and ensure the buffers are of the desired length and type, direct
+   * buffers or not.
+   * @param buffers the buffers to check
+   */
+  void checkOutputBuffers(ByteBuffer[] buffers) {
+    for (ByteBuffer buffer : buffers) {
+      if (buffer == null) {
+        throw new IllegalArgumentException(
+            "Invalid buffer found, not allowing null");
+      }
+
+      if (buffer.remaining() != decodeLength) {
+        throw new IllegalArgumentException(
+            "Invalid buffer, not of length " + decodeLength);
+      }
+      if (buffer.isDirect() != usingDirectBuffer) {
+        throw new IllegalArgumentException(
+            "Invalid buffer, isDirect should be " + usingDirectBuffer);
+      }
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferEncodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferEncodingState.java
new file mode 100644
index 0000000..4ca7a3f
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferEncodingState.java
@@ -0,0 +1,110 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A utility class that maintains encoding state during an encode call using
+ * ByteBuffer inputs.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class ByteBufferEncodingState extends EncodingState {
+  ByteBuffer[] inputs;
+  ByteBuffer[] outputs;
+  boolean usingDirectBuffer;
+
+  ByteBufferEncodingState(RawErasureEncoder encoder,
+                          ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    this.encoder = encoder;
+    ByteBuffer validInput = CoderUtil.findFirstValidInput(inputs);
+    this.encodeLength = validInput.remaining();
+    this.usingDirectBuffer = validInput.isDirect();
+    this.inputs = inputs;
+    this.outputs = outputs;
+
+    checkParameters(inputs, outputs);
+    checkBuffers(inputs);
+    checkBuffers(outputs);
+  }
+
+  ByteBufferEncodingState(RawErasureEncoder encoder,
+                          int encodeLength,
+                          ByteBuffer[] inputs,
+                          ByteBuffer[] outputs) {
+    this.encoder = encoder;
+    this.encodeLength = encodeLength;
+    this.inputs = inputs;
+    this.outputs = outputs;
+  }
+
+  /**
+   * Convert to a ByteArrayEncodingState when it's backed by on-heap arrays.
+   */
+  ByteArrayEncodingState convertToByteArrayState() {
+    int[] inputOffsets = new int[inputs.length];
+    int[] outputOffsets = new int[outputs.length];
+    byte[][] newInputs = new byte[inputs.length][];
+    byte[][] newOutputs = new byte[outputs.length][];
+
+    ByteBuffer buffer;
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      inputOffsets[i] = buffer.arrayOffset() + buffer.position();
+      newInputs[i] = buffer.array();
+    }
+
+    for (int i = 0; i < outputs.length; ++i) {
+      buffer = outputs[i];
+      outputOffsets[i] = buffer.arrayOffset() + buffer.position();
+      newOutputs[i] = buffer.array();
+    }
+
+    ByteArrayEncodingState baeState = new ByteArrayEncodingState(encoder,
+        encodeLength, newInputs, inputOffsets, newOutputs, outputOffsets);
+    return baeState;
+  }
+
+  /**
+   * Check and ensure the buffers are of the desired length and type, direct
+   * buffers or not.
+   * @param buffers the buffers to check
+   */
+  void checkBuffers(ByteBuffer[] buffers) {
+    for (ByteBuffer buffer : buffers) {
+      if (buffer == null) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer found, not allowing null");
+      }
+
+      if (buffer.remaining() != encodeLength) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer remaining " + buffer.remaining()
+                + ", not of length " + encodeLength);
+      }
+      if (buffer.isDirect() != usingDirectBuffer) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer, isDirect should be " + usingDirectBuffer);
+      }
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/CoderUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/CoderUtil.java
new file mode 100644
index 0000000..04bb980
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/CoderUtil.java
@@ -0,0 +1,174 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.ozone.erasurecode.ECChunk;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Helpful utilities for implementing some raw erasure coders.
+ */
+public final class CoderUtil {
+
+  private CoderUtil() {
+    // No called
+  }
+
+  private static byte[] emptyChunk = new byte[4096];
+
+  /**
+   * Make sure to return an empty chunk buffer for the desired length.
+   * @param leastLength
+   * @return empty chunk of zero bytes
+   */
+  static byte[] getEmptyChunk(int leastLength) {
+    if (emptyChunk.length >= leastLength) {
+      return emptyChunk; // In most time
+    }
+
+    synchronized (CoderUtil.class) {
+      emptyChunk = new byte[leastLength];
+    }
+
+    return emptyChunk;
+  }
+
+  /**
+   * Ensure a buffer filled with ZERO bytes from current readable/writable
+   * position.
+   * @param buffer a buffer ready to read / write certain size bytes
+   * @return the buffer itself, with ZERO bytes written, the position and limit
+   *         are not changed after the call
+   */
+  static ByteBuffer resetBuffer(ByteBuffer buffer, int len) {
+    int pos = buffer.position();
+    buffer.put(getEmptyChunk(len), 0, len);
+    buffer.position(pos);
+
+    return buffer;
+  }
+
+  /**
+   * Ensure the buffer (either input or output) ready to read or write with ZERO
+   * bytes fully in specified length of len.
+   * @param buffer bytes array buffer
+   * @return the buffer itself
+   */
+  static byte[] resetBuffer(byte[] buffer, int offset, int len) {
+    byte[] empty = getEmptyChunk(len);
+    System.arraycopy(empty, 0, buffer, offset, len);
+
+    return buffer;
+  }
+
+  /**
+   * Initialize the output buffers with ZERO bytes.
+   */
+  static void resetOutputBuffers(ByteBuffer[] buffers, int dataLen) {
+    for (ByteBuffer buffer : buffers) {
+      resetBuffer(buffer, dataLen);
+    }
+  }
+
+  /**
+   * Initialize the output buffers with ZERO bytes.
+   */
+  static void resetOutputBuffers(byte[][] buffers, int[] offsets,
+                                 int dataLen) {
+    for (int i = 0; i < buffers.length; i++) {
+      resetBuffer(buffers[i], offsets[i], dataLen);
+    }
+  }
+
+
+  /**
+   * Convert an array of this chunks to an array of ByteBuffers.
+   *
+   * @param chunks chunks to convertToByteArrayState into buffers
+   * @return an array of ByteBuffers
+   */
+  static ByteBuffer[] toBuffers(ECChunk[] chunks) {
+    ByteBuffer[] buffers = new ByteBuffer[chunks.length];
+
+    ECChunk chunk;
+    for (int i = 0; i < chunks.length; i++) {
+      chunk = chunks[i];
+      if (chunk == null) {
+        buffers[i] = null;
+      } else {
+        buffers[i] = chunk.getBuffer();
+        if (chunk.isAllZero()) {
+          CoderUtil.resetBuffer(buffers[i], buffers[i].remaining());
+        }
+      }
+    }
+
+    return buffers;
+  }
+
+  /**
+   * Clone an input bytes array as direct ByteBuffer.
+   */
+  static ByteBuffer cloneAsDirectByteBuffer(byte[] input, int offset, int len) {
+    if (input == null) { // an input can be null, if erased or not to read
+      return null;
+    }
+
+    ByteBuffer directBuffer = ByteBuffer.allocateDirect(len);
+    directBuffer.put(input, offset, len);
+    directBuffer.flip();
+    return directBuffer;
+  }
+
+
+
+  /**
+   * Find the valid input from all the inputs.
+   * @param inputs input buffers to look for valid input
+   * @return the first valid input
+   */
+  static <T> T findFirstValidInput(T[] inputs) {
+    for (T input : inputs) {
+      if (input != null) {
+        return input;
+      }
+    }
+
+    throw new IllegalArgumentException(
+        "Invalid inputs are found, all being null");
+  }
+
+  /**
+   * Picking up indexes of valid inputs.
+   * @param inputs decoding input buffers
+   * @param <T>
+   */
+  static <T> int[] getValidIndexes(T[] inputs) {
+    int[] validIndexes = new int[inputs.length];
+    int idx = 0;
+    for (int i = 0; i < inputs.length; i++) {
+      if (inputs[i] != null) {
+        validIndexes[idx++] = i;
+      }
+    }
+
+    return Arrays.copyOf(validIndexes, idx);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DecodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DecodingState.java
new file mode 100644
index 0000000..7a407b8
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DecodingState.java
@@ -0,0 +1,52 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+/**
+ * A utility class that maintains decoding state during a decode call.
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class DecodingState {
+  RawErasureDecoder decoder;
+  int decodeLength;
+
+  /**
+   * Check and validate decoding parameters, throw exception accordingly. The
+   * checking assumes it's a MDS code. Other code  can override this.
+   * @param inputs input buffers to check
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs output buffers to check
+   */
+  <T> void checkParameters(T[] inputs, int[] erasedIndexes,
+                           T[] outputs) {
+    if (inputs.length != decoder.getNumParityUnits() +
+        decoder.getNumDataUnits()) {
+      throw new IllegalArgumentException("Invalid inputs length");
+    }
+
+    if (erasedIndexes.length != outputs.length) {
+      throw new IllegalArgumentException(
+          "erasedIndexes and outputs mismatch in length");
+    }
+
+    if (erasedIndexes.length > decoder.getNumParityUnits()) {
+      throw new IllegalArgumentException(
+          "Too many erased, not recoverable");
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawDecoder.java
new file mode 100644
index 0000000..4a11dbc
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawDecoder.java
@@ -0,0 +1,45 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A dummy raw decoder that does no real computation.
+ * Instead, it just returns zero bytes.
+ * This decoder can be used to isolate the performance issue to HDFS side logic
+ * instead of codec, and is intended for test only.
+ */
+@InterfaceAudience.Private
+public class DummyRawDecoder extends RawErasureDecoder {
+
+  public DummyRawDecoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+  }
+
+  @Override
+  protected void doDecode(ByteBufferDecodingState decodingState) {
+    // Nothing to do. Output buffers have already been reset
+  }
+
+  @Override
+  protected void doDecode(ByteArrayDecodingState decodingState) {
+    // Nothing to do. Output buffers have already been reset
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawEncoder.java
new file mode 100644
index 0000000..9b178e6
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawEncoder.java
@@ -0,0 +1,45 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A dummy raw encoder that does no real computation.
+ * Instead, it just returns zero bytes.
+ * This encoder can be used to isolate the performance issue to HDFS side logic
+ * instead of codec, and is intended for test only.
+ */
+@InterfaceAudience.Private
+public class DummyRawEncoder extends RawErasureEncoder {
+
+  public DummyRawEncoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+  }
+
+  @Override
+  protected void doEncode(ByteArrayEncodingState encodingState) {
+    // Nothing to do. Output buffers have already been reset
+  }
+
+  @Override
+  protected void doEncode(ByteBufferEncodingState encodingState) {
+    // Nothing to do. Output buffers have already been reset
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawErasureCoderFactory.java
new file mode 100644
index 0000000..bf74871
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawErasureCoderFactory.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw erasure coder factory for dummy raw coders.
+ */
+@InterfaceAudience.Private
+public class DummyRawErasureCoderFactory implements RawErasureCoderFactory {
+  public static final String CODER_NAME = "dummy_dummy";
+  public static final String DUMMY_CODEC_NAME = "dummy";
+
+  @Override
+  public RawErasureEncoder createEncoder(
+      ECReplicationConfig ecReplicationConfig) {
+    return new DummyRawEncoder(ecReplicationConfig);
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder(
+      ECReplicationConfig ecReplicationConfig) {
+    return new DummyRawDecoder(ecReplicationConfig);
+  }
+
+  @Override
+  public String getCoderName() {
+    return CODER_NAME;
+  }
+
+  @Override
+  public String getCodecName() {
+    return DUMMY_CODEC_NAME;
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/EncodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/EncodingState.java
new file mode 100644
index 0000000..2339342
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/EncodingState.java
@@ -0,0 +1,47 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A utility class that maintains encoding state during an encode call.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+abstract class EncodingState {
+  RawErasureEncoder encoder;
+  int encodeLength;
+
+  /**
+   * Check and validate decoding parameters, throw exception accordingly.
+   * @param inputs input buffers to check
+   * @param outputs output buffers to check
+   */
+  <T> void checkParameters(T[] inputs, T[] outputs) {
+    if (inputs.length != encoder.getNumDataUnits()) {
+      throw new HadoopIllegalArgumentException("Invalid inputs length "
+          + inputs.length + " !=" + encoder.getNumDataUnits());
+    }
+    if (outputs.length != encoder.getNumParityUnits()) {
+      throw new HadoopIllegalArgumentException("Invalid outputs length "
+          + outputs.length + " !=" + encoder.getNumParityUnits());
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ErasureCodeNative.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ErasureCodeNative.java
new file mode 100644
index 0000000..4fcac54
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ErasureCodeNative.java
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Erasure code native libraries (for now, Intel ISA-L) related utilities.
+ *
+ * Note: This class should be modified to check native ozone libs loaded once
+ * ozone native libs started to include native erasure coded bits. For now, we
+ * will just depend on libhadoop lib to check whether native erasure coded libs
+ * available.
+ */
+public final class ErasureCodeNative {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ErasureCodeNative.class.getName());
+
+  /**
+   * The reason why ISA-L library is not available, or null if it is available.
+   */
+  private static final String LOADING_FAILURE_REASON;
+
+  static {
+    if (!NativeCodeLoader.isNativeCodeLoaded()) {
+      LOADING_FAILURE_REASON = "hadoop native library cannot be loaded.";
+    } else if (!NativeCodeLoader.buildSupportsIsal()) {
+      LOADING_FAILURE_REASON = "libhadoop was built without ISA-L support";
+    } else {
+      String problem = null;
+      try {
+        loadLibrary();
+      } catch (Throwable t) {
+        problem = "Loading ISA-L failed: " + t.getMessage();
+        LOG.warn(problem, t);
+      }
+      LOADING_FAILURE_REASON = problem;
+    }
+
+    if (LOADING_FAILURE_REASON != null) {
+      LOG.warn("ISA-L support is not available in your platform... " +
+            "using builtin-java codec where applicable");
+    }
+  }
+
+  private ErasureCodeNative() {
+  }
+
+  /**
+   * Are native libraries loaded?
+   */
+  public static boolean isNativeCodeLoaded() {
+    return LOADING_FAILURE_REASON == null;
+  }
+
+  /**
+   * Is the native ISA-L library loaded and initialized? Throw exception if not.
+   */
+  public static void checkNativeCodeLoaded() {
+    if (LOADING_FAILURE_REASON != null) {
+      throw new RuntimeException(LOADING_FAILURE_REASON);
+    }
+  }
+
+  /**
+   * Load native library available or supported.
+   */
+  public static native void loadLibrary();
+
+  /**
+   * Get the native library name that's available or supported.
+   */
+  public static native String getLibraryName();
+
+  public static String getLoadingFailureReason() {
+    return LOADING_FAILURE_REASON;
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawDecoder.java
new file mode 100644
index 0000000..98c5e8e
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawDecoder.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.HadoopNativeECAccessorUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A Reed-Solomon raw decoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeRSRawDecoder extends AbstractNativeRawDecoder {
+
+  private org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawDecoder
+      hadoopNativeRSRawDecoder;
+
+  public NativeRSRawDecoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+    hadoopNativeRSRawDecoder =
+        new org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawDecoder(
+            new ErasureCoderOptions(ecReplicationConfig.getData(),
+                ecReplicationConfig.getParity()));
+  }
+
+  @Override
+  protected void performDecodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+      int dataLen, int[] erased, ByteBuffer[] outputs, int[] outputOffsets)
+      throws IOException {
+    HadoopNativeECAccessorUtil
+        .performDecodeImpl(hadoopNativeRSRawDecoder, inputs,
+            inputOffsets, dataLen, erased, outputs, outputOffsets);
+  }
+
+  @Override
+  public void release() {
+    hadoopNativeRSRawDecoder.release();
+  }
+
+  @Override
+  public boolean preferDirectBuffer() {
+    return hadoopNativeRSRawDecoder.preferDirectBuffer();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawEncoder.java
new file mode 100644
index 0000000..62db61e
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawEncoder.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.HadoopNativeECAccessorUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A Reed-Solomon raw encoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeRSRawEncoder extends AbstractNativeRawEncoder {
+
+  private org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawEncoder
+      hadoopNativeRSRawEncoder;
+
+  public NativeRSRawEncoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+    hadoopNativeRSRawEncoder =
+        new org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawEncoder(
+            new ErasureCoderOptions(ecReplicationConfig.getData(),
+                ecReplicationConfig.getParity()));
+  }
+
+  @Override
+  protected void performEncodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+      int dataLen, ByteBuffer[] outputs, int[] outputOffsets)
+      throws IOException {
+    HadoopNativeECAccessorUtil
+        .performEncodeImpl(hadoopNativeRSRawEncoder, inputs,
+            inputOffsets, dataLen, outputs, outputOffsets);
+  }
+
+  @Override
+  public void release() {
+    hadoopNativeRSRawEncoder.release();
+  }
+
+  @Override
+  public boolean preferDirectBuffer() {
+    return hadoopNativeRSRawEncoder.preferDirectBuffer();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawErasureCoderFactory.java
new file mode 100644
index 0000000..b2088ec
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawErasureCoderFactory.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw coder factory for raw Reed-Solomon coder in native using Intel ISA-L.
+ */
+@InterfaceAudience.Private
+public class NativeRSRawErasureCoderFactory
+    implements RawErasureCoderFactory {
+
+  public static final String CODER_NAME = "rs_native";
+
+  @Override
+  public RawErasureEncoder createEncoder(
+      ECReplicationConfig ecReplicationConfig) {
+    return new NativeRSRawEncoder(ecReplicationConfig);
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder(
+      ECReplicationConfig ecReplicationConfig) {
+    return new NativeRSRawDecoder(ecReplicationConfig);
+  }
+
+  @Override
+  public String getCoderName() {
+    return CODER_NAME;
+  }
+
+  @Override
+  public String getCodecName() {
+    return ECReplicationConfig.EcCodec.RS.name().toLowerCase();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawDecoder.java
new file mode 100644
index 0000000..96f8581
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawDecoder.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.HadoopNativeECAccessorUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A XOR raw decoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeXORRawDecoder extends AbstractNativeRawDecoder {
+
+  private org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawDecoder
+      hadoopNativeXORRawDecoder;
+
+  public NativeXORRawDecoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+    hadoopNativeXORRawDecoder =
+        new org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawDecoder(
+            new ErasureCoderOptions(ecReplicationConfig.getData(),
+                ecReplicationConfig.getParity()));
+  }
+
+  @Override
+  protected void performDecodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+      int dataLen, int[] erased, ByteBuffer[] outputs, int[] outputOffsets)
+      throws IOException {
+    HadoopNativeECAccessorUtil
+        .performDecodeImpl(hadoopNativeXORRawDecoder, inputs,
+            inputOffsets, dataLen, erased, outputs, outputOffsets);
+  }
+
+  @Override
+  public void release() {
+    hadoopNativeXORRawDecoder.release();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawEncoder.java
new file mode 100644
index 0000000..91be762
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawEncoder.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.HadoopNativeECAccessorUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A XOR raw encoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeXORRawEncoder extends AbstractNativeRawEncoder {
+
+  private org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawEncoder
+      hadoopNativeXORRawEncoder;
+
+  public NativeXORRawEncoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+    hadoopNativeXORRawEncoder =
+        new org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawEncoder(
+            new ErasureCoderOptions(ecReplicationConfig.getData(),
+                ecReplicationConfig.getParity()));
+  }
+
+  @Override
+  protected void performEncodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+      int dataLen, ByteBuffer[] outputs, int[] outputOffsets)
+      throws IOException {
+    HadoopNativeECAccessorUtil
+        .performEncodeImpl(hadoopNativeXORRawEncoder, inputs,
+            inputOffsets, dataLen, outputs, outputOffsets);
+  }
+
+  @Override
+  public void release() {
+    hadoopNativeXORRawEncoder.release();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawErasureCoderFactory.java
new file mode 100644
index 0000000..4a2eb50
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawErasureCoderFactory.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw coder factory for xor coder in native using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeXORRawErasureCoderFactory
+    implements RawErasureCoderFactory {
+
+  public static final String CODER_NAME = "xor_native";
+
+  @Override
+  public RawErasureEncoder createEncoder(
+      ECReplicationConfig ecReplicationConfig) {
+    return new NativeXORRawEncoder(ecReplicationConfig);
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder(
+      ECReplicationConfig ecReplicationConfig) {
+    return new NativeXORRawDecoder(ecReplicationConfig);
+  }
+
+  @Override
+  public String getCoderName() {
+    return CODER_NAME;
+  }
+
+  @Override
+  public String getCodecName() {
+    return ECReplicationConfig.EcCodec.XOR.name().toLowerCase();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawDecoder.java
new file mode 100644
index 0000000..a61e1ea
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawDecoder.java
@@ -0,0 +1,177 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.rawcoder.util.DumpUtil;
+import org.apache.ozone.erasurecode.rawcoder.util.GF256;
+import org.apache.ozone.erasurecode.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * A raw erasure decoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible. This new Java coder is about 5X faster than the one originated
+ * from HDFS-RAID, and also compatible with the native/ISA-L coder.
+ */
+@InterfaceAudience.Private
+public class RSRawDecoder extends RawErasureDecoder {
+  //relevant to schema and won't change during decode calls
+  private byte[] encodeMatrix;
+
+  /**
+   * Below are relevant to schema and erased indexes, thus may change during
+   * decode calls.
+   */
+  private byte[] decodeMatrix;
+  private byte[] invertMatrix;
+  /**
+   * Array of input tables generated from coding coefficients previously.
+   * Must be of size 32*k*rows
+   */
+  private byte[] gfTables;
+  private int[] cachedErasedIndexes;
+  private int[] validIndexes;
+  private int numErasedDataUnits;
+  private boolean[] erasureFlags;
+
+  public RSRawDecoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+
+    int numAllUnits = getNumAllUnits();
+    if (getNumAllUnits() >= RSUtil.GF.getFieldSize()) {
+      throw new HadoopIllegalArgumentException(
+              "Invalid getNumDataUnits() and numParityUnits");
+    }
+
+    encodeMatrix = new byte[numAllUnits * getNumDataUnits()];
+    RSUtil.genCauchyMatrix(encodeMatrix, numAllUnits, getNumDataUnits());
+    if (allowVerboseDump()) {
+      DumpUtil.dumpMatrix(encodeMatrix, getNumDataUnits(), numAllUnits);
+    }
+  }
+
+  @Override
+  protected void doDecode(ByteBufferDecodingState decodingState) {
+    CoderUtil.resetOutputBuffers(decodingState.outputs,
+        decodingState.decodeLength);
+    prepareDecoding(decodingState.inputs, decodingState.erasedIndexes);
+
+    ByteBuffer[] realInputs = new ByteBuffer[getNumDataUnits()];
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      realInputs[i] = decodingState.inputs[validIndexes[i]];
+    }
+    RSUtil.encodeData(gfTables, realInputs, decodingState.outputs);
+  }
+
+  @Override
+  protected void doDecode(ByteArrayDecodingState decodingState) {
+    int dataLen = decodingState.decodeLength;
+    CoderUtil.resetOutputBuffers(decodingState.outputs,
+        decodingState.outputOffsets, dataLen);
+    prepareDecoding(decodingState.inputs, decodingState.erasedIndexes);
+
+    byte[][] realInputs = new byte[getNumDataUnits()][];
+    int[] realInputOffsets = new int[getNumDataUnits()];
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      realInputs[i] = decodingState.inputs[validIndexes[i]];
+      realInputOffsets[i] = decodingState.inputOffsets[validIndexes[i]];
+    }
+    RSUtil.encodeData(gfTables, dataLen, realInputs, realInputOffsets,
+        decodingState.outputs, decodingState.outputOffsets);
+  }
+
+  private <T> void prepareDecoding(T[] inputs, int[] erasedIndexes) {
+    int[] tmpValidIndexes = CoderUtil.getValidIndexes(inputs);
+    if (Arrays.equals(this.cachedErasedIndexes, erasedIndexes) &&
+        Arrays.equals(this.validIndexes, tmpValidIndexes)) {
+      return; // Optimization. Nothing to do
+    }
+    this.cachedErasedIndexes =
+            Arrays.copyOf(erasedIndexes, erasedIndexes.length);
+    this.validIndexes =
+            Arrays.copyOf(tmpValidIndexes, tmpValidIndexes.length);
+
+    processErasures(erasedIndexes);
+  }
+
+  private void processErasures(int[] erasedIndexes) {
+    this.decodeMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+    this.invertMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+    this.gfTables = new byte[getNumAllUnits() * getNumDataUnits() * 32];
+
+    this.erasureFlags = new boolean[getNumAllUnits()];
+    this.numErasedDataUnits = 0;
+
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      int index = erasedIndexes[i];
+      erasureFlags[index] = true;
+      if (index < getNumDataUnits()) {
+        numErasedDataUnits++;
+      }
+    }
+
+    generateDecodeMatrix(erasedIndexes);
+
+    RSUtil.initTables(getNumDataUnits(), erasedIndexes.length,
+        decodeMatrix, 0, gfTables);
+    if (allowVerboseDump()) {
+      System.out.println(DumpUtil.bytesToHex(gfTables, -1));
+    }
+  }
+
+  // Generate decode matrix from encode matrix
+  private void generateDecodeMatrix(int[] erasedIndexes) {
+    int i, j, r, p;
+    byte s;
+    byte[] tmpMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+
+    // Construct matrix tmpMatrix by removing error rows
+    for (i = 0; i < getNumDataUnits(); i++) {
+      r = validIndexes[i];
+      for (j = 0; j < getNumDataUnits(); j++) {
+        tmpMatrix[getNumDataUnits() * i + j] =
+                encodeMatrix[getNumDataUnits() * r + j];
+      }
+    }
+
+    GF256.gfInvertMatrix(tmpMatrix, invertMatrix, getNumDataUnits());
+
+    for (i = 0; i < numErasedDataUnits; i++) {
+      for (j = 0; j < getNumDataUnits(); j++) {
+        decodeMatrix[getNumDataUnits() * i + j] =
+                invertMatrix[getNumDataUnits() * erasedIndexes[i] + j];
+      }
+    }
+
+    for (p = numErasedDataUnits; p < erasedIndexes.length; p++) {
+      for (i = 0; i < getNumDataUnits(); i++) {
+        s = 0;
+        for (j = 0; j < getNumDataUnits(); j++) {
+          s ^= GF256.gfMul(invertMatrix[j * getNumDataUnits() + i],
+                  encodeMatrix[getNumDataUnits() * erasedIndexes[p] + j]);
+        }
+        decodeMatrix[getNumDataUnits() * p + i] = s;
+      }
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawEncoder.java
new file mode 100644
index 0000000..086b1c5
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawEncoder.java
@@ -0,0 +1,77 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.rawcoder.util.DumpUtil;
+import org.apache.ozone.erasurecode.rawcoder.util.RSUtil;
+
+/**
+ * A raw erasure encoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible. This new Java coder is about 5X faster than the one originated
+ * from HDFS-RAID, and also compatible with the native/ISA-L coder.
+ */
+public class RSRawEncoder extends RawErasureEncoder {
+  // relevant to schema and won't change during encode calls.
+  private byte[] encodeMatrix;
+  /**
+   * Array of input tables generated from coding coefficients previously.
+   * Must be of size 32*k*rows
+   */
+  private byte[] gfTables;
+
+  public RSRawEncoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+
+    if (getNumAllUnits() >= RSUtil.GF.getFieldSize()) {
+      throw new IllegalArgumentException(
+          "Invalid numDataUnits and numParityUnits");
+    }
+
+    encodeMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+    RSUtil.genCauchyMatrix(encodeMatrix, getNumAllUnits(), getNumDataUnits());
+    if (allowVerboseDump()) {
+      DumpUtil.dumpMatrix(encodeMatrix, getNumDataUnits(), getNumAllUnits());
+    }
+    gfTables = new byte[getNumAllUnits() * getNumDataUnits() * 32];
+    RSUtil.initTables(getNumDataUnits(), getNumParityUnits(), encodeMatrix,
+        getNumDataUnits() * getNumDataUnits(), gfTables);
+    if (allowVerboseDump()) {
+      System.out.println(DumpUtil.bytesToHex(gfTables, -1));
+    }
+  }
+
+  @Override
+  protected void doEncode(ByteBufferEncodingState encodingState) {
+    CoderUtil.resetOutputBuffers(encodingState.outputs,
+        encodingState.encodeLength);
+    RSUtil.encodeData(gfTables, encodingState.inputs, encodingState.outputs);
+  }
+
+  @Override
+  protected void doEncode(ByteArrayEncodingState encodingState) {
+    CoderUtil.resetOutputBuffers(encodingState.outputs,
+        encodingState.outputOffsets,
+        encodingState.encodeLength);
+    RSUtil.encodeData(gfTables, encodingState.encodeLength,
+        encodingState.inputs,
+        encodingState.inputOffsets, encodingState.outputs,
+        encodingState.outputOffsets);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawErasureCoderFactory.java
new file mode 100644
index 0000000..ece5d9d
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawErasureCoderFactory.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw coder factory for the new raw Reed-Solomon coder in Java.
+ */
+@InterfaceAudience.Private
+public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  public static final String CODER_NAME = "rs_java";
+
+  @Override
+  public RawErasureEncoder createEncoder(
+      ECReplicationConfig ecReplicationConfig) {
+    return new RSRawEncoder(ecReplicationConfig);
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder(
+      ECReplicationConfig ecReplicationConfig) {
+    return new RSRawDecoder(ecReplicationConfig);
+  }
+
+  @Override
+  public String getCoderName() {
+    return CODER_NAME;
+  }
+
+  @Override
+  public String getCodecName() {
+    return ECReplicationConfig.EcCodec.RS.name().toLowerCase();
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderFactory.java
new file mode 100644
index 0000000..27cc049
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderFactory.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * Raw erasure coder factory that can be used to create raw encoder and decoder.
+ * It helps in configuration since only one factory class is needed to be
+ * configured.
+ */
+@InterfaceAudience.Private
+public interface RawErasureCoderFactory {
+
+  /**
+   * Create raw erasure encoder.
+   * @param ecReplicationConfig the config used to create the encoder
+   * @return raw erasure encoder
+   */
+  RawErasureEncoder createEncoder(ECReplicationConfig ecReplicationConfig);
+
+  /**
+   * Create raw erasure decoder.
+   * @param ecReplicationConfig the config used to create the encoder
+   * @return raw erasure decoder
+   */
+  RawErasureDecoder createDecoder(ECReplicationConfig ecReplicationConfig);
+
+  /**
+   * Get the name of the coder.
+   * @return coder name
+   */
+  String getCoderName();
+
+  /**
+   * Get the name of its codec.
+   * @return codec name
+   */
+  String getCodecName();
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureDecoder.java
new file mode 100644
index 0000000..144ba41
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureDecoder.java
@@ -0,0 +1,217 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.ECChunk;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure decoder that's to be inherited by new decoders.
+ *
+ * Raw erasure coder is part of erasure codec framework, where erasure coder is
+ * used to encode/decode a group of blocks (BlockGroup) according to the codec
+ * specific BlockGroup layout and logic. An erasure coder extracts chunks of
+ * data from the blocks and can employ various low level raw erasure coders to
+ * perform encoding/decoding against the chunks.
+ *
+ * To distinguish from erasure coder, here raw erasure coder is used to mean the
+ * low level constructs, since it only takes care of the math calculation with
+ * a group of byte buffers.
+ *
+ * Note it mainly provides decode() calls, which should be stateless and may be
+ * made thread-safe in future.
+ */
+public abstract class RawErasureDecoder {
+
+  private final ECReplicationConfig coderOptions;
+
+  public RawErasureDecoder(ECReplicationConfig coderOptions) {
+    this.coderOptions = coderOptions;
+  }
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * How to prepare for inputs:
+   * 1. Create an array containing data units + parity units. Please note the
+   *    data units should be first or before the parity units.
+   * 2. Set null in the array locations specified via erasedIndexes to indicate
+   *    they're erased and no data are to read from;
+   * 3. Set null in the array locations for extra redundant items, as they're
+   *    not necessary to read when decoding. For example in RS-6-3, if only 1
+   *    unit is really erased, then we have 2 extra items as redundant. They can
+   *    be set as null to indicate no data will be used from them.
+   *
+   * For an example using RS (6, 3), assuming sources (d0, d1, d2, d3, d4, d5)
+   * and parities (p0, p1, p2), d2 being erased. We can and may want to use only
+   * 6 units like (d1, d3, d4, d5, p0, p2) to recover d2. We will have:
+   *     inputs = [null(d0), d1, null(d2), d3, d4, d5, p0, null(p1), p2]
+   *     erasedIndexes = [2] // index of d2 into inputs array
+   *     outputs = [a-writable-buffer]
+   *
+   * Note, for both inputs and outputs, no mixing of on-heap buffers and direct
+   * buffers are allowed.
+   *
+   * If the coder option ALLOW_CHANGE_INPUTS is set true (false by default), the
+   * content of input buffers may change after the call, subject to concrete
+   * implementation.
+   *
+   * @param inputs input buffers to read data from. The buffers' remaining will
+   *               be 0 after decoding
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs output buffers to put decoded data into according to
+   *                erasedIndexes, ready for read after the call
+   */
+  public synchronized void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+                     ByteBuffer[] outputs) throws IOException {
+    ByteBufferDecodingState decodingState = new ByteBufferDecodingState(this,
+        inputs, erasedIndexes, outputs);
+
+    boolean usingDirectBuffer = decodingState.usingDirectBuffer;
+    int dataLen = decodingState.decodeLength;
+    if (dataLen == 0) {
+      return;
+    }
+
+    int[] inputPositions = new int[inputs.length];
+    for (int i = 0; i < inputPositions.length; i++) {
+      if (inputs[i] != null) {
+        inputPositions[i] = inputs[i].position();
+      }
+    }
+
+    if (usingDirectBuffer) {
+      doDecode(decodingState);
+    } else {
+      ByteArrayDecodingState badState = decodingState.convertToByteArrayState();
+      doDecode(badState);
+    }
+
+    for (int i = 0; i < inputs.length; i++) {
+      if (inputs[i] != null) {
+        // dataLen bytes consumed
+        inputs[i].position(inputPositions[i] + dataLen);
+      }
+    }
+  }
+
+  /**
+   * Perform the real decoding using Direct ByteBuffer.
+   * @param decodingState the decoding state
+   */
+  protected abstract void doDecode(ByteBufferDecodingState decodingState)
+      throws IOException;
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs. More see above.
+   *
+   * @param inputs input buffers to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs output buffers to put decoded data into according to
+   *                erasedIndexes, ready for read after the call
+   * @throws IOException if the decoder is closed.
+   */
+  public synchronized void decode(byte[][] inputs, int[] erasedIndexes,
+      byte[][] outputs) throws IOException {
+    ByteArrayDecodingState decodingState = new ByteArrayDecodingState(this,
+        inputs, erasedIndexes, outputs);
+
+    if (decodingState.decodeLength == 0) {
+      return;
+    }
+
+    doDecode(decodingState);
+  }
+
+  /**
+   * Perform the real decoding using bytes array, supporting offsets and
+   * lengths.
+   * @param decodingState the decoding state
+   * @throws IOException if the decoder is closed.
+   */
+  protected abstract void doDecode(ByteArrayDecodingState decodingState)
+      throws IOException;
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs. More see above.
+   *
+   * Note, for both input and output ECChunks, no mixing of on-heap buffers and
+   * direct buffers are allowed.
+   *
+   * @param inputs input buffers to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs output buffers to put decoded data into according to
+   *                erasedIndexes, ready for read after the call
+   * @throws IOException if the decoder is closed
+   */
+  public synchronized void decode(ECChunk[] inputs, int[] erasedIndexes,
+                     ECChunk[] outputs) throws IOException {
+    ByteBuffer[] newInputs = CoderUtil.toBuffers(inputs);
+    ByteBuffer[] newOutputs = CoderUtil.toBuffers(outputs);
+    decode(newInputs, erasedIndexes, newOutputs);
+  }
+
+  public int getNumDataUnits() {
+    return coderOptions.getData();
+  }
+
+  public int getNumParityUnits() {
+    return coderOptions.getParity();
+  }
+
+  protected int getNumAllUnits() {
+    return coderOptions.getData() + coderOptions.getParity();
+  }
+
+  /**
+   * Tell if direct buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, using DirectByteBuffer or
+   * bytes array. It will return false by default.
+   * @return true if native buffer is preferred for performance consideration,
+   * otherwise false.
+   */
+  public boolean preferDirectBuffer() {
+    return false;
+  }
+
+  /**
+   * Allow change into input buffers or not while perform encoding/decoding.
+   * @return true if it's allowed to change inputs, false otherwise
+   */
+  public boolean allowChangeInputs() {
+    return false;
+  }
+
+  /**
+   * Allow to dump verbose info during encoding/decoding.
+   * @return true if it's allowed to do verbose dump, false otherwise.
+   */
+  public boolean allowVerboseDump() {
+    return false;
+  }
+
+  /**
+   * Should be called when release this coder. Good chance to release encoding
+   * or decoding buffers
+   */
+  public void release() {
+    // Nothing to do here.
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureEncoder.java
new file mode 100644
index 0000000..2d3e704
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureEncoder.java
@@ -0,0 +1,193 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.ECChunk;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure encoder that's to be inherited by new encoders.
+ *
+ * Raw erasure coder is part of erasure codec framework, where erasure coder is
+ * used to encode/decode a group of blocks (BlockGroup) according to the codec
+ * specific BlockGroup layout and logic. An erasure coder extracts chunks of
+ * data from the blocks and can employ various low level raw erasure coders to
+ * perform encoding/decoding against the chunks.
+ *
+ * To distinguish from erasure coder, here raw erasure coder is used to mean the
+ * low level constructs, since it only takes care of the math calculation with
+ * a group of byte buffers.
+ *
+ * Note it mainly provides encode() calls, which should be stateless and may be
+ * made thread-safe in future.
+ */
+public abstract class RawErasureEncoder {
+
+  private final ECReplicationConfig coderOptions;
+
+  public RawErasureEncoder(ECReplicationConfig coderOptions) {
+    this.coderOptions = coderOptions;
+  }
+
+  /**
+   * Encode with inputs and generates outputs.
+   *
+   * Note, for both inputs and outputs, no mixing of on-heap buffers and direct
+   * buffers are allowed.
+   *
+   * If the coder option ALLOW_CHANGE_INPUTS is set true (false by default), the
+   * content of input buffers may change after the call, subject to concrete
+   * implementation. Anyway the positions of input buffers will move forward.
+   *
+   * @param inputs input buffers to read data from. The buffers' remaining will
+   *               be 0 after encoding
+   * @param outputs output buffers to put the encoded data into, ready to read
+   *                after the call
+   * @throws IOException if the encoder is closed.
+   */
+  public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs)
+      throws IOException {
+    ByteBufferEncodingState bbeState = new ByteBufferEncodingState(
+        this, inputs, outputs);
+
+    boolean usingDirectBuffer = bbeState.usingDirectBuffer;
+    int dataLen = bbeState.encodeLength;
+    if (dataLen == 0) {
+      return;
+    }
+
+    int[] inputPositions = new int[inputs.length];
+    for (int i = 0; i < inputPositions.length; i++) {
+      if (inputs[i] != null) {
+        inputPositions[i] = inputs[i].position();
+      }
+    }
+
+    if (usingDirectBuffer) {
+      doEncode(bbeState);
+    } else {
+      ByteArrayEncodingState baeState = bbeState.convertToByteArrayState();
+      doEncode(baeState);
+    }
+
+    for (int i = 0; i < inputs.length; i++) {
+      if (inputs[i] != null) {
+        // dataLen bytes consumed
+        inputs[i].position(inputPositions[i] + dataLen);
+      }
+    }
+  }
+
+  /**
+   * Perform the real encoding work using direct ByteBuffer.
+   * @param encodingState the encoding state
+   */
+  protected abstract void doEncode(ByteBufferEncodingState encodingState)
+      throws IOException;
+
+  /**
+   * Encode with inputs and generates outputs. More see above.
+   *
+   * @param inputs input buffers to read data from
+   * @param outputs output buffers to put the encoded data into, read to read
+   *                after the call
+   */
+  public void encode(byte[][] inputs, byte[][] outputs) throws IOException {
+    ByteArrayEncodingState baeState = new ByteArrayEncodingState(
+        this, inputs, outputs);
+
+    int dataLen = baeState.encodeLength;
+    if (dataLen == 0) {
+      return;
+    }
+
+    doEncode(baeState);
+  }
+
+  /**
+   * Perform the real encoding work using bytes array, supporting offsets
+   * and lengths.
+   * @param encodingState the encoding state
+   */
+  protected abstract void doEncode(ByteArrayEncodingState encodingState)
+      throws IOException;
+
+  /**
+   * Encode with inputs and generates outputs. More see above.
+   *
+   * @param inputs input buffers to read data from
+   * @param outputs output buffers to put the encoded data into, read to read
+   *                after the call
+   * @throws IOException if the encoder is closed.
+   */
+  public void encode(ECChunk[] inputs, ECChunk[] outputs) throws IOException {
+    ByteBuffer[] newInputs = ECChunk.toBuffers(inputs);
+    ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs);
+    encode(newInputs, newOutputs);
+  }
+
+  public int getNumDataUnits() {
+    return coderOptions.getData();
+  }
+
+  public int getNumParityUnits() {
+    return coderOptions.getParity();
+  }
+
+  public int getNumAllUnits() {
+    return coderOptions.getData() + coderOptions.getParity();
+  }
+
+  /**
+   * Tell if direct buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, using DirectByteBuffer or
+   * bytes array. It will return false by default.
+   * @return true if native buffer is preferred for performance consideration,
+   * otherwise false.
+   */
+  public boolean preferDirectBuffer() {
+    return false;
+  }
+
+  /**
+   * Allow change into input buffers or not while perform encoding/decoding.
+   * @return true if it's allowed to change inputs, false otherwise
+   */
+  public boolean allowChangeInputs() {
+    return false;
+  }
+
+  /**
+   * Allow to dump verbose info during encoding/decoding.
+   * @return true if it's allowed to do verbose dump, false otherwise.
+   */
+  public boolean allowVerboseDump() {
+    return false;
+  }
+
+  /**
+   * Should be called when release this coder. Good chance to release encoding
+   * or decoding buffers
+   */
+  public void release() {
+    // Nothing to do here.
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawDecoder.java
new file mode 100644
index 0000000..1829b17
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawDecoder.java
@@ -0,0 +1,88 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ *
+ * XOR code is an important primitive code scheme in erasure coding and often
+ * used in advanced codes, like HitchHiker and LRC, though itself is rarely
+ * deployed independently.
+ */
+@InterfaceAudience.Private
+public class XORRawDecoder extends RawErasureDecoder {
+
+  public XORRawDecoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+  }
+
+  @Override
+  protected void doDecode(ByteBufferDecodingState decodingState) {
+    CoderUtil.resetOutputBuffers(decodingState.outputs,
+        decodingState.decodeLength);
+    ByteBuffer output = decodingState.outputs[0];
+
+    int erasedIdx = decodingState.erasedIndexes[0];
+
+    // Process the inputs.
+    int iIdx, oIdx;
+    for (int i = 0; i < decodingState.inputs.length; i++) {
+      // Skip the erased location.
+      if (i == erasedIdx) {
+        continue;
+      }
+
+      for (iIdx = decodingState.inputs[i].position(), oIdx = output.position();
+           iIdx < decodingState.inputs[i].limit();
+           iIdx++, oIdx++) {
+        output.put(oIdx, (byte) (output.get(oIdx) ^
+            decodingState.inputs[i].get(iIdx)));
+      }
+    }
+  }
+
+  @Override
+  protected void doDecode(ByteArrayDecodingState decodingState) {
+    byte[] output = decodingState.outputs[0];
+    int dataLen = decodingState.decodeLength;
+    CoderUtil.resetOutputBuffers(decodingState.outputs,
+        decodingState.outputOffsets, dataLen);
+    int erasedIdx = decodingState.erasedIndexes[0];
+
+    // Process the inputs.
+    int iIdx, oIdx;
+    for (int i = 0; i < decodingState.inputs.length; i++) {
+      // Skip the erased location.
+      if (i == erasedIdx) {
+        continue;
+      }
+
+      for (iIdx = decodingState.inputOffsets[i],
+               oIdx = decodingState.outputOffsets[0];
+           iIdx < decodingState.inputOffsets[i] + dataLen; iIdx++, oIdx++) {
+        output[oIdx] ^= decodingState.inputs[i][iIdx];
+      }
+    }
+  }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawEncoder.java
new file mode 100644
index 0000000..c02e97c
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawEncoder.java
@@ -0,0 +1,86 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ *
+ * XOR code is an important primitive code scheme in erasure coding and often
+ * used in advanced codes, like HitchHiker and LRC, though itself is rarely
+ * deployed independently.
+ */
+@InterfaceAudience.Private
+public class XORRawEncoder extends RawErasureEncoder {
+
+  public XORRawEncoder(ECReplicationConfig ecReplicationConfig) {
+    super(ecReplicationConfig);
+  }
+
+  protected void doEncode(ByteBufferEncodingState encodingState) {
+    CoderUtil.resetOutputBuffers(encodingState.outputs,
+        encodingState.encodeLength);
+    ByteBuffer output = encodingState.outputs[0];
+
+    // Get the first buffer's data.
+    int iIdx, oIdx;
+    for (iIdx = encodingState.inputs[0].position(), oIdx = output.position();
+         iIdx < encodingState.inputs[0].limit(); iIdx++, oIdx++) {
+      output.put(oIdx, encodingState.inputs[0].get(iIdx));
+    }
+
+    // XOR with everything else.
+    for (int i = 1; i < encodingState.inputs.length; i++) {
+      for (iIdx = encodingState.inputs[i].position(), oIdx = output.position();
+           iIdx < encodingState.inputs[i].limit();
+           iIdx++, oIdx++) {
+        output.put(oIdx, (byte) (output.get(oIdx) ^
+            encodingState.inputs[i].get(iIdx)));
+      }
+    }
+  }
+
+  @Override
+  protected void doEncode(ByteArrayEncodingState encodingState) {
+    int dataLen = encodingState.encodeLength;
+    CoderUtil.resetOutputBuffers(encodingState.outputs,
+        encodingState.outputOffsets, dataLen);
+    byte[] output = encodingState.outputs[0];
+
+    // Get the first buffer's data.
+    int iIdx, oIdx;
+    for (iIdx = encodingState.inputOffsets[0],
+             oIdx = encodingState.outputOffsets[0];
+         iIdx < encodingState.inputOffsets[0] + dataLen; iIdx++, oIdx++) {
+      output[oIdx] = encodingState.inputs[0][iIdx];
+    }
+
+    // XOR with everything else.
+    for (int i = 1; i < encodingState.inputs.length; i++) {
+      for (iIdx = encodingState.inputOffsets[i],
+               oIdx = encodingState.outputOffsets[0];
+           iIdx < encodingState.inputOffsets[i] + dataLen; iIdx++, oIdx++) {
+        output[oIdx] ^= encodingState.inputs[i][iIdx];
+      }
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawErasureCoderFactory.java
new file mode 100644
index 0000000..631af5b
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawErasureCoderFactory.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw coder factory for raw XOR coder.
+ */
+@InterfaceAudience.Private
+public class XORRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  public static final String CODER_NAME = "xor_java";
+
+
+  @Override
+  public RawErasureEncoder createEncoder(
+      ECReplicationConfig replicationConfig) {
+    return new XORRawEncoder(replicationConfig);
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder(
+      ECReplicationConfig replicationConfig) {
+    return new XORRawDecoder(replicationConfig);
+  }
+
+  @Override
+  public String getCoderName() {
+    return CODER_NAME;
+  }
+
+  @Override
+  public String getCodecName() {
+    return ECReplicationConfig.EcCodec.XOR.name().toLowerCase();
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/package-info.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/package-info.java
new file mode 100644
index 0000000..cfad4a3
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/package-info.java
@@ -0,0 +1,38 @@
+/**
+ * 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.
+ */
+
+/**
+ *
+ * Raw erasure coders.
+ *
+ * Raw erasure coder is part of erasure codec framework, where erasure coder is
+ * used to encode/decode a group of blocks (BlockGroup) according to the codec
+ * specific BlockGroup layout and logic. An erasure coder extracts chunks of
+ * data from the blocks and can employ various low level raw erasure coders to
+ * perform encoding/decoding against the chunks.
+ *
+ * To distinguish from erasure coder, here raw erasure coder is used to mean the
+ * low level constructs, since it only takes care of the math calculation with
+ * a group of byte buffers.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/CodecUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/CodecUtil.java
new file mode 100644
index 0000000..4b4ab41
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/CodecUtil.java
@@ -0,0 +1,111 @@
+/**
+ * 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.ozone.erasurecode.rawcoder.util;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A codec &amp; coder utility to help create coders conveniently.
+ * <p>
+ * {@link CodecUtil} includes erasure coder configurations key and default
+ * values such as coder class name and erasure codec option values included
+ * by {@link ECReplicationConfig}.{@link RawErasureEncoder} and
+ * {@link RawErasureDecoder} are created by createRawEncoder and
+ * createRawDecoder.
+ */
+@InterfaceAudience.Private
+public final class CodecUtil {
+
+  public static final Logger LOG = LoggerFactory.getLogger(CodecUtil.class);
+
+  private CodecUtil() {
+  }
+
+  private static RawErasureCoderFactory createRawCoderFactory(String coderName,
+      String codecName) {
+    RawErasureCoderFactory fact;
+    fact = CodecRegistry.getInstance().
+        getCoderByName(codecName, coderName);
+
+    return fact;
+  }
+
+  public static RawErasureEncoder createRawEncoderWithFallback(
+      final ECReplicationConfig ecReplicationConfig) {
+    // Note: Coders can be configurable, but for now, we just use whats
+    //  available.
+    String codecName = ecReplicationConfig.getCodec().name().toLowerCase();
+    String[] rawCoderNames =
+        CodecRegistry.getInstance().getCoderNames(codecName);
+    for (String rawCoderName : rawCoderNames) {
+      try {
+        if (rawCoderName != null) {
+          RawErasureCoderFactory fact =
+              createRawCoderFactory(rawCoderName, codecName);
+          return fact.createEncoder(ecReplicationConfig);
+        }
+      } catch (LinkageError | Exception e) {
+        // Fallback to next coder if possible
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "Failed to create raw erasure encoder " + rawCoderName
+                  + ", fallback to next codec if possible",
+              e);
+        }
+      }
+    }
+    throw new IllegalArgumentException(
+        "Fail to create raw erasure " + "encoder with given codec: "
+            + codecName);
+  }
+
+  public static RawErasureDecoder createRawDecoderWithFallback(
+      final ECReplicationConfig ecReplicationConfig) {
+    // Note: Coders can be configurable, but for now, we just use whats
+    //  available.
+    String codecName = ecReplicationConfig.getCodec().name().toLowerCase();
+    String[] coders = CodecRegistry.getInstance().getCoderNames(codecName);
+    for (String rawCoderName : coders) {
+      try {
+        if (rawCoderName != null) {
+          RawErasureCoderFactory fact =
+              createRawCoderFactory(rawCoderName, codecName);
+          return fact.createDecoder(ecReplicationConfig);
+        }
+      } catch (LinkageError | Exception e) {
+        // Fallback to next coder if possible
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "Failed to create raw erasure decoder " + rawCoderName
+                  + ", fallback to next codec if possible",
+              e);
+        }
+      }
+    }
+    throw new IllegalArgumentException(
+        "Fail to create raw erasure " + "decoder with given codec: "
+            + codecName);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/DumpUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/DumpUtil.java
new file mode 100644
index 0000000..54e9d76
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/DumpUtil.java
@@ -0,0 +1,99 @@
+/**
+ * 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.ozone.erasurecode.rawcoder.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.ozone.erasurecode.ECChunk;
+
+/**
+ * A dump utility class for debugging data erasure coding/decoding issues.
+ * Don't suggest they are used in runtime production codes.
+ */
+@InterfaceAudience.Private
+public final class DumpUtil {
+  private static final String HEX_CHARS_STR = "0123456789ABCDEF";
+  private static final char[] HEX_CHARS = HEX_CHARS_STR.toCharArray();
+
+  private DumpUtil() {
+    // No called
+  }
+
+  /**
+   * Convert bytes into format like 0x02 02 00 80.
+   * If limit is negative or too large, then all bytes will be converted.
+   */
+  public static String bytesToHex(byte[] bytes, int limit) {
+    if (limit <= 0 || limit > bytes.length) {
+      limit = bytes.length;
+    }
+    int len = limit * 2;
+    len += limit; // for ' ' appended for each char
+    len += 2; // for '0x' prefix
+    char[] hexChars = new char[len];
+    hexChars[0] = '0';
+    hexChars[1] = 'x';
+    for (int j = 0; j < limit; j++) {
+      int v = bytes[j] & 0xFF;
+      hexChars[j * 3 + 2] = HEX_CHARS[v >>> 4];
+      hexChars[j * 3 + 3] = HEX_CHARS[v & 0x0F];
+      hexChars[j * 3 + 4] = ' ';
+    }
+
+    return new String(hexChars);
+  }
+
+  public static void dumpMatrix(byte[] matrix,
+                                int numDataUnits, int numAllUnits) {
+    for (int i = 0; i < numDataUnits; i++) {
+      for (int j = 0; j < numAllUnits; j++) {
+        System.out.print(" ");
+        System.out.print(0xff & matrix[i * numAllUnits + j]);
+      }
+      System.out.println();
+    }
+  }
+
+  /**
+   * Print data in hex format in an array of chunks.
+   * @param header
+   * @param chunks
+   */
+  public static void dumpChunks(String header, ECChunk[] chunks) {
+    System.out.println();
+    System.out.println(header);
+    for (int i = 0; i < chunks.length; i++) {
+      dumpChunk(chunks[i]);
+    }
+    System.out.println();
+  }
+
+  /**
+   * Print data in hex format in a chunk.
+   * @param chunk
+   */
+  public static void dumpChunk(ECChunk chunk) {
+    String str;
+    if (chunk == null) {
+      str = "<EMPTY>";
+    } else {
+      byte[] bytes = chunk.toBytesArray();
+      str = DumpUtil.bytesToHex(bytes, 16);
+    }
+    System.out.println(str);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GF256.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GF256.java
new file mode 100644
index 0000000..4e0b2d2
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GF256.java
@@ -0,0 +1,333 @@
+/**
+ * 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.ozone.erasurecode.rawcoder.util;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A GaloisField utility class only caring of 256 fields for efficiency. Some
+ * of the codes are borrowed from ISA-L implementation (C or ASM codes).
+ */
+@InterfaceAudience.Private
+public final class GF256 {
+
+  private GF256() { }
+
+  private static final byte[] GF_BASE = new byte[]{
+      (byte) 0x01, (byte) 0x02, (byte) 0x04, (byte) 0x08, (byte) 0x10,
+      (byte) 0x20, (byte) 0x40, (byte) 0x80, (byte) 0x1d, (byte) 0x3a,
+      (byte) 0x74, (byte) 0xe8, (byte) 0xcd, (byte) 0x87, (byte) 0x13,
+      (byte) 0x26, (byte) 0x4c, (byte) 0x98, (byte) 0x2d, (byte) 0x5a,
+      (byte) 0xb4, (byte) 0x75, (byte) 0xea, (byte) 0xc9, (byte) 0x8f,
+      (byte) 0x03, (byte) 0x06, (byte) 0x0c, (byte) 0x18, (byte) 0x30,
+      (byte) 0x60, (byte) 0xc0, (byte) 0x9d, (byte) 0x27, (byte) 0x4e,
+      (byte) 0x9c, (byte) 0x25, (byte) 0x4a, (byte) 0x94, (byte) 0x35,
+      (byte) 0x6a, (byte) 0xd4, (byte) 0xb5, (byte) 0x77, (byte) 0xee,
+      (byte) 0xc1, (byte) 0x9f, (byte) 0x23, (byte) 0x46, (byte) 0x8c,
+      (byte) 0x05, (byte) 0x0a, (byte) 0x14, (byte) 0x28, (byte) 0x50,
+      (byte) 0xa0, (byte) 0x5d, (byte) 0xba, (byte) 0x69, (byte) 0xd2,
+      (byte) 0xb9, (byte) 0x6f, (byte) 0xde, (byte) 0xa1, (byte) 0x5f,
+      (byte) 0xbe, (byte) 0x61, (byte) 0xc2, (byte) 0x99, (byte) 0x2f,
+      (byte) 0x5e, (byte) 0xbc, (byte) 0x65, (byte) 0xca, (byte) 0x89,
+      (byte) 0x0f, (byte) 0x1e, (byte) 0x3c, (byte) 0x78, (byte) 0xf0,
+      (byte) 0xfd, (byte) 0xe7, (byte) 0xd3, (byte) 0xbb, (byte) 0x6b,
+      (byte) 0xd6, (byte) 0xb1, (byte) 0x7f, (byte) 0xfe, (byte) 0xe1,
+      (byte) 0xdf, (byte) 0xa3, (byte) 0x5b, (byte) 0xb6, (byte) 0x71,
+      (byte) 0xe2, (byte) 0xd9, (byte) 0xaf, (byte) 0x43, (byte) 0x86,
+      (byte) 0x11, (byte) 0x22, (byte) 0x44, (byte) 0x88, (byte) 0x0d,
+      (byte) 0x1a, (byte) 0x34, (byte) 0x68, (byte) 0xd0, (byte) 0xbd,
+      (byte) 0x67, (byte) 0xce, (byte) 0x81, (byte) 0x1f, (byte) 0x3e,
+      (byte) 0x7c, (byte) 0xf8, (byte) 0xed, (byte) 0xc7, (byte) 0x93,
+      (byte) 0x3b, (byte) 0x76, (byte) 0xec, (byte) 0xc5, (byte) 0x97,
+      (byte) 0x33, (byte) 0x66, (byte) 0xcc, (byte) 0x85, (byte) 0x17,
+      (byte) 0x2e, (byte) 0x5c, (byte) 0xb8, (byte) 0x6d, (byte) 0xda,
+      (byte) 0xa9, (byte) 0x4f, (byte) 0x9e, (byte) 0x21, (byte) 0x42,
+      (byte) 0x84, (byte) 0x15, (byte) 0x2a, (byte) 0x54, (byte) 0xa8,
+      (byte) 0x4d, (byte) 0x9a, (byte) 0x29, (byte) 0x52, (byte) 0xa4,
+      (byte) 0x55, (byte) 0xaa, (byte) 0x49, (byte) 0x92, (byte) 0x39,
+      (byte) 0x72, (byte) 0xe4, (byte) 0xd5, (byte) 0xb7, (byte) 0x73,
+      (byte) 0xe6, (byte) 0xd1, (byte) 0xbf, (byte) 0x63, (byte) 0xc6,
+      (byte) 0x91, (byte) 0x3f, (byte) 0x7e, (byte) 0xfc, (byte) 0xe5,
+      (byte) 0xd7, (byte) 0xb3, (byte) 0x7b, (byte) 0xf6, (byte) 0xf1,
+      (byte) 0xff, (byte) 0xe3, (byte) 0xdb, (byte) 0xab, (byte) 0x4b,
+      (byte) 0x96, (byte) 0x31, (byte) 0x62, (byte) 0xc4, (byte) 0x95,
+      (byte) 0x37, (byte) 0x6e, (byte) 0xdc, (byte) 0xa5, (byte) 0x57,
+      (byte) 0xae, (byte) 0x41, (byte) 0x82, (byte) 0x19, (byte) 0x32,
+      (byte) 0x64, (byte) 0xc8, (byte) 0x8d, (byte) 0x07, (byte) 0x0e,
+      (byte) 0x1c, (byte) 0x38, (byte) 0x70, (byte) 0xe0, (byte) 0xdd,
+      (byte) 0xa7, (byte) 0x53, (byte) 0xa6, (byte) 0x51, (byte) 0xa2,
+      (byte) 0x59, (byte) 0xb2, (byte) 0x79, (byte) 0xf2, (byte) 0xf9,
+      (byte) 0xef, (byte) 0xc3, (byte) 0x9b, (byte) 0x2b, (byte) 0x56,
+      (byte) 0xac, (byte) 0x45, (byte) 0x8a, (byte) 0x09, (byte) 0x12,
+      (byte) 0x24, (byte) 0x48, (byte) 0x90, (byte) 0x3d, (byte) 0x7a,
+      (byte) 0xf4, (byte) 0xf5, (byte) 0xf7, (byte) 0xf3, (byte) 0xfb,
+      (byte) 0xeb, (byte) 0xcb, (byte) 0x8b, (byte) 0x0b, (byte) 0x16,
+      (byte) 0x2c, (byte) 0x58, (byte) 0xb0, (byte) 0x7d, (byte) 0xfa,
+      (byte) 0xe9, (byte) 0xcf, (byte) 0x83, (byte) 0x1b, (byte) 0x36,
+      (byte) 0x6c, (byte) 0xd8, (byte) 0xad, (byte) 0x47, (byte) 0x8e,
+      (byte) 0x01
+  };
+
+  private static final byte[] GF_LOG_BASE = new byte[]{
+      (byte) 0x00, (byte) 0xff, (byte) 0x01, (byte) 0x19, (byte) 0x02,
+      (byte) 0x32, (byte) 0x1a, (byte) 0xc6, (byte) 0x03, (byte) 0xdf,
+      (byte) 0x33, (byte) 0xee, (byte) 0x1b, (byte) 0x68, (byte) 0xc7,
+      (byte) 0x4b, (byte) 0x04, (byte) 0x64, (byte) 0xe0, (byte) 0x0e,
+      (byte) 0x34, (byte) 0x8d, (byte) 0xef, (byte) 0x81, (byte) 0x1c,
+      (byte) 0xc1, (byte) 0x69, (byte) 0xf8, (byte) 0xc8, (byte) 0x08,
+      (byte) 0x4c, (byte) 0x71, (byte) 0x05, (byte) 0x8a, (byte) 0x65,
+      (byte) 0x2f, (byte) 0xe1, (byte) 0x24, (byte) 0x0f, (byte) 0x21,
+      (byte) 0x35, (byte) 0x93, (byte) 0x8e, (byte) 0xda, (byte) 0xf0,
+      (byte) 0x12, (byte) 0x82, (byte) 0x45, (byte) 0x1d, (byte) 0xb5,
+      (byte) 0xc2, (byte) 0x7d, (byte) 0x6a, (byte) 0x27, (byte) 0xf9,
+      (byte) 0xb9, (byte) 0xc9, (byte) 0x9a, (byte) 0x09, (byte) 0x78,
+      (byte) 0x4d, (byte) 0xe4, (byte) 0x72, (byte) 0xa6, (byte) 0x06,
+      (byte) 0xbf, (byte) 0x8b, (byte) 0x62, (byte) 0x66, (byte) 0xdd,
+      (byte) 0x30, (byte) 0xfd, (byte) 0xe2, (byte) 0x98, (byte) 0x25,
+      (byte) 0xb3, (byte) 0x10, (byte) 0x91, (byte) 0x22, (byte) 0x88,
+      (byte) 0x36, (byte) 0xd0, (byte) 0x94, (byte) 0xce, (byte) 0x8f,
+      (byte) 0x96, (byte) 0xdb, (byte) 0xbd, (byte) 0xf1, (byte) 0xd2,
+      (byte) 0x13, (byte) 0x5c, (byte) 0x83, (byte) 0x38, (byte) 0x46,
+      (byte) 0x40, (byte) 0x1e, (byte) 0x42, (byte) 0xb6, (byte) 0xa3,
+      (byte) 0xc3, (byte) 0x48, (byte) 0x7e, (byte) 0x6e, (byte) 0x6b,
+      (byte) 0x3a, (byte) 0x28, (byte) 0x54, (byte) 0xfa, (byte) 0x85,
+      (byte) 0xba, (byte) 0x3d, (byte) 0xca, (byte) 0x5e, (byte) 0x9b,
+      (byte) 0x9f, (byte) 0x0a, (byte) 0x15, (byte) 0x79, (byte) 0x2b,
+      (byte) 0x4e, (byte) 0xd4, (byte) 0xe5, (byte) 0xac, (byte) 0x73,
+      (byte) 0xf3, (byte) 0xa7, (byte) 0x57, (byte) 0x07, (byte) 0x70,
+      (byte) 0xc0, (byte) 0xf7, (byte) 0x8c, (byte) 0x80, (byte) 0x63,
+      (byte) 0x0d, (byte) 0x67, (byte) 0x4a, (byte) 0xde, (byte) 0xed,
+      (byte) 0x31, (byte) 0xc5, (byte) 0xfe, (byte) 0x18, (byte) 0xe3,
+      (byte) 0xa5, (byte) 0x99, (byte) 0x77, (byte) 0x26, (byte) 0xb8,
+      (byte) 0xb4, (byte) 0x7c, (byte) 0x11, (byte) 0x44, (byte) 0x92,
+      (byte) 0xd9, (byte) 0x23, (byte) 0x20, (byte) 0x89, (byte) 0x2e,
+      (byte) 0x37, (byte) 0x3f, (byte) 0xd1, (byte) 0x5b, (byte) 0x95,
+      (byte) 0xbc, (byte) 0xcf, (byte) 0xcd, (byte) 0x90, (byte) 0x87,
+      (byte) 0x97, (byte) 0xb2, (byte) 0xdc, (byte) 0xfc, (byte) 0xbe,
+      (byte) 0x61, (byte) 0xf2, (byte) 0x56, (byte) 0xd3, (byte) 0xab,
+      (byte) 0x14, (byte) 0x2a, (byte) 0x5d, (byte) 0x9e, (byte) 0x84,
+      (byte) 0x3c, (byte) 0x39, (byte) 0x53, (byte) 0x47, (byte) 0x6d,
+      (byte) 0x41, (byte) 0xa2, (byte) 0x1f, (byte) 0x2d, (byte) 0x43,
+      (byte) 0xd8, (byte) 0xb7, (byte) 0x7b, (byte) 0xa4, (byte) 0x76,
+      (byte) 0xc4, (byte) 0x17, (byte) 0x49, (byte) 0xec, (byte) 0x7f,
+      (byte) 0x0c, (byte) 0x6f, (byte) 0xf6, (byte) 0x6c, (byte) 0xa1,
+      (byte) 0x3b, (byte) 0x52, (byte) 0x29, (byte) 0x9d, (byte) 0x55,
+      (byte) 0xaa, (byte) 0xfb, (byte) 0x60, (byte) 0x86, (byte) 0xb1,
+      (byte) 0xbb, (byte) 0xcc, (byte) 0x3e, (byte) 0x5a, (byte) 0xcb,
+      (byte) 0x59, (byte) 0x5f, (byte) 0xb0, (byte) 0x9c, (byte) 0xa9,
+      (byte) 0xa0, (byte) 0x51, (byte) 0x0b, (byte) 0xf5, (byte) 0x16,
+      (byte) 0xeb, (byte) 0x7a, (byte) 0x75, (byte) 0x2c, (byte) 0xd7,
+      (byte) 0x4f, (byte) 0xae, (byte) 0xd5, (byte) 0xe9, (byte) 0xe6,
+      (byte) 0xe7, (byte) 0xad, (byte) 0xe8, (byte) 0x74, (byte) 0xd6,
+      (byte) 0xf4, (byte) 0xea, (byte) 0xa8, (byte) 0x50, (byte) 0x58,
+      (byte) 0xaf
+  };
+
+  @SuppressFBWarnings("MS_EXPOSE_REP")
+  private static byte[][] theGfMulTab; // multiply result table in GF 256 space
+
+  /**
+   * Initialize the GF multiply table for performance. Just compute once, and
+   * avoid repeatedly doing the multiply during encoding/decoding.
+   */
+  static {
+    theGfMulTab = new byte[256][256];
+    for (int i = 0; i < 256; i++) {
+      for (int j = 0; j < 256; j++) {
+        theGfMulTab[i][j] = gfMul((byte) i, (byte) j);
+      }
+    }
+  }
+
+  /**
+   * Get the big GF multiply table so utilize it efficiently.
+   * @return the big GF multiply table
+   */
+  public static byte[][] gfMulTab() {
+    return theGfMulTab;
+  }
+
+  public static byte gfMul(byte a, byte b) {
+    if ((a == 0) || (b == 0)) {
+      return 0;
+    }
+
+    int tmp = (GF_LOG_BASE[a & 0xff] & 0xff) +
+        (GF_LOG_BASE[b & 0xff] & 0xff);
+    if (tmp > 254) {
+      tmp -= 255;
+    }
+
+    return GF_BASE[tmp];
+  }
+
+  public static byte gfInv(byte a) {
+    if (a == 0) {
+      return 0;
+    }
+
+    return GF_BASE[255 - GF_LOG_BASE[a & 0xff] & 0xff];
+  }
+
+  /**
+   * Invert a matrix assuming it's invertible.
+   *
+   * Ported from Intel ISA-L library.
+   */
+  public static void gfInvertMatrix(byte[] inMatrix, byte[] outMatrix, int n) {
+    byte temp;
+
+    // Set outMatrix[] to the identity matrix
+    for (int i = 0; i < n * n; i++) {
+      // memset(outMatrix, 0, n*n)
+      outMatrix[i] = 0;
+    }
+
+    for (int i = 0; i < n; i++) {
+      outMatrix[i * n + i] = 1;
+    }
+
+    // Inverse
+    for (int j, i = 0; i < n; i++) {
+      // Check for 0 in pivot element
+      if (inMatrix[i * n + i] == 0) {
+        // Find a row with non-zero in current column and swap
+        for (j = i + 1; j < n; j++) {
+          if (inMatrix[j * n + i] != 0) {
+            break;
+          }
+        }
+        if (j == n) {
+          // Couldn't find means it's singular
+          throw new RuntimeException("Not invertible");
+        }
+
+        for (int k = 0; k < n; k++) {
+          // Swap rows i,j
+          temp = inMatrix[i * n + k];
+          inMatrix[i * n + k] = inMatrix[j * n + k];
+          inMatrix[j * n + k] = temp;
+
+          temp = outMatrix[i * n + k];
+          outMatrix[i * n + k] = outMatrix[j * n + k];
+          outMatrix[j * n + k] = temp;
+        }
+      }
+
+      temp = gfInv(inMatrix[i * n + i]); // 1/pivot
+      for (j = 0; j < n; j++) {
+        // Scale row i by 1/pivot
+        inMatrix[i * n + j] = gfMul(inMatrix[i * n + j], temp);
+        outMatrix[i * n + j] = gfMul(outMatrix[i * n + j], temp);
+      }
+
+      for (j = 0; j < n; j++) {
+        if (j == i) {
+          continue;
+        }
+
+        temp = inMatrix[j * n + i];
+        for (int k = 0; k < n; k++) {
+          outMatrix[j * n + k] ^= gfMul(temp, outMatrix[i * n + k]);
+          inMatrix[j * n + k] ^= gfMul(temp, inMatrix[i * n + k]);
+        }
+      }
+    }
+  }
+
+  /**
+   * Ported from Intel ISA-L library.
+   *
+   * Calculates const table gftbl in GF(2^8) from single input A
+   * gftbl(A) = {A{00}, A{01}, A{02}, ... , A{0f} }, {A{00}, A{10}, A{20},
+   * ... , A{f0} } -- from ISA-L implementation
+   */
+  public static void gfVectMulInit(byte c, byte[] tbl, int offset) {
+    byte c2 = (byte) ((c << 1) ^ ((c & 0x80) != 0 ? 0x1d : 0));
+    byte c4 = (byte) ((c2 << 1) ^ ((c2 & 0x80) != 0 ? 0x1d : 0));
+    byte c8 = (byte) ((c4 << 1) ^ ((c4 & 0x80) != 0 ? 0x1d : 0));
+
+    byte c3, c5, c6, c7, c9, c10, c11, c12, c13, c14, c15;
+    byte c17, c18, c19, c20, c21, c22, c23, c24, c25, c26,
+        c27, c28, c29, c30, c31;
+
+    c3 = (byte) (c2 ^ c);
+    c5 = (byte) (c4 ^ c);
+    c6 = (byte) (c4 ^ c2);
+    c7 = (byte) (c4 ^ c3);
+
+    c9 = (byte) (c8 ^ c);
+    c10 = (byte) (c8 ^ c2);
+    c11 = (byte) (c8 ^ c3);
+    c12 = (byte) (c8 ^ c4);
+    c13 = (byte) (c8 ^ c5);
+    c14 = (byte) (c8 ^ c6);
+    c15 = (byte) (c8 ^ c7);
+
+    tbl[offset + 0] = 0;
+    tbl[offset + 1] = c;
+    tbl[offset + 2] = c2;
+    tbl[offset + 3] = c3;
+    tbl[offset + 4] = c4;
+    tbl[offset + 5] = c5;
+    tbl[offset + 6] = c6;
+    tbl[offset + 7] = c7;
+    tbl[offset + 8] = c8;
+    tbl[offset + 9] = c9;
+    tbl[offset + 10] = c10;
+    tbl[offset + 11] = c11;
+    tbl[offset + 12] = c12;
+    tbl[offset + 13] = c13;
+    tbl[offset + 14] = c14;
+    tbl[offset + 15] = c15;
+
+    c17 = (byte) ((c8 << 1) ^ ((c8 & 0x80) != 0 ? 0x1d : 0));
+    c18 = (byte) ((c17 << 1) ^ ((c17 & 0x80) != 0 ? 0x1d : 0));
+    c19 = (byte) (c18 ^ c17);
+    c20 = (byte) ((c18 << 1) ^ ((c18 & 0x80) != 0 ? 0x1d : 0));
+    c21 = (byte) (c20 ^ c17);
+    c22 = (byte) (c20 ^ c18);
+    c23 = (byte) (c20 ^ c19);
+    c24 = (byte) ((c20 << 1) ^ ((c20 & 0x80) != 0 ? 0x1d : 0));
+    c25 = (byte) (c24 ^ c17);
+    c26 = (byte) (c24 ^ c18);
+    c27 = (byte) (c24 ^ c19);
+    c28 = (byte) (c24 ^ c20);
+    c29 = (byte) (c24 ^ c21);
+    c30 = (byte) (c24 ^ c22);
+    c31 = (byte) (c24 ^ c23);
+
+    tbl[offset + 16] = 0;
+    tbl[offset + 17] = c17;
+    tbl[offset + 18] = c18;
+    tbl[offset + 19] = c19;
+    tbl[offset + 20] = c20;
+    tbl[offset + 21] = c21;
+    tbl[offset + 22] = c22;
+    tbl[offset + 23] = c23;
+    tbl[offset + 24] = c24;
+    tbl[offset + 25] = c25;
+    tbl[offset + 26] = c26;
+    tbl[offset + 27] = c27;
+    tbl[offset + 28] = c28;
+    tbl[offset + 29] = c29;
+    tbl[offset + 30] = c30;
+    tbl[offset + 31] = c31;
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GaloisField.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GaloisField.java
new file mode 100644
index 0000000..7fb06ce
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GaloisField.java
@@ -0,0 +1,565 @@
+/**
+ * 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.ozone.erasurecode.rawcoder.util;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Implementation of Galois field arithmetic with 2^p elements. The input must
+ * be unsigned integers. It's ported from HDFS-RAID, slightly adapted.
+ */
+@InterfaceAudience.Private
+public final class GaloisField {
+
+  // Field size 256 is good for byte based system
+  private static final int DEFAULT_FIELD_SIZE = 256;
+  // primitive polynomial 1 + X^2 + X^3 + X^4 + X^8 (substitute 2)
+  private static final int DEFAULT_PRIMITIVE_POLYNOMIAL = 285;
+  private static final Map<Integer, GaloisField> INSTANCES =
+      new HashMap<>();
+  private final int[] logTable;
+  private final int[] powTable;
+  private final int[][] mulTable;
+  private final int[][] divTable;
+  private final int fieldSize;
+  private final int primitivePeriod;
+  private final int primitivePolynomial;
+
+  private GaloisField(int fieldSize, int primitivePolynomial) {
+    assert fieldSize > 0;
+    assert primitivePolynomial > 0;
+
+    this.fieldSize = fieldSize;
+    this.primitivePeriod = fieldSize - 1;
+    this.primitivePolynomial = primitivePolynomial;
+    logTable = new int[fieldSize];
+    powTable = new int[fieldSize];
+    mulTable = new int[fieldSize][fieldSize];
+    divTable = new int[fieldSize][fieldSize];
+    int value = 1;
+    for (int pow = 0; pow < fieldSize - 1; pow++) {
+      powTable[pow] = value;
+      logTable[value] = pow;
+      value = value * 2;
+      if (value >= fieldSize) {
+        value = value ^ primitivePolynomial;
+      }
+    }
+    // building multiplication table
+    for (int i = 0; i < fieldSize; i++) {
+      for (int j = 0; j < fieldSize; j++) {
+        if (i == 0 || j == 0) {
+          mulTable[i][j] = 0;
+          continue;
+        }
+        int z = logTable[i] + logTable[j];
+        z = z >= primitivePeriod ? z - primitivePeriod : z;
+        z = powTable[z];
+        mulTable[i][j] = z;
+      }
+    }
+    // building division table
+    for (int i = 0; i < fieldSize; i++) {
+      for (int j = 1; j < fieldSize; j++) {
+        if (i == 0) {
+          divTable[i][j] = 0;
+          continue;
+        }
+        int z = logTable[i] - logTable[j];
+        z = z < 0 ? z + primitivePeriod : z;
+        z = powTable[z];
+        divTable[i][j] = z;
+      }
+    }
+  }
+
+  /**
+   * Get the object performs Galois field arithmetics.
+   *
+   * @param fieldSize           size of the field
+   * @param primitivePolynomial a primitive polynomial corresponds to the size
+   */
+  public static GaloisField getInstance(int fieldSize,
+                                        int primitivePolynomial) {
+    int key = ((fieldSize << 16) & 0xFFFF0000)
+        + (primitivePolynomial & 0x0000FFFF);
+    GaloisField gf;
+    synchronized (INSTANCES) {
+      gf = INSTANCES.get(key);
+      if (gf == null) {
+        gf = new GaloisField(fieldSize, primitivePolynomial);
+        INSTANCES.put(key, gf);
+      }
+    }
+    return gf;
+  }
+
+  /**
+   * Get the object performs Galois field arithmetic with default setting.
+   */
+  public static GaloisField getInstance() {
+    return getInstance(DEFAULT_FIELD_SIZE, DEFAULT_PRIMITIVE_POLYNOMIAL);
+  }
+
+  /**
+   * Return number of elements in the field.
+   *
+   * @return number of elements in the field
+   */
+  public int getFieldSize() {
+    return fieldSize;
+  }
+
+  /**
+   * Return the primitive polynomial in GF(2).
+   *
+   * @return primitive polynomial as a integer
+   */
+  public int getPrimitivePolynomial() {
+    return primitivePolynomial;
+  }
+
+  /**
+   * Compute the sum of two fields.
+   *
+   * @param x input field
+   * @param y input field
+   * @return result of addition
+   */
+  public int add(int x, int y) {
+    assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize());
+    return x ^ y;
+  }
+
+  /**
+   * Compute the multiplication of two fields.
+   *
+   * @param x input field
+   * @param y input field
+   * @return result of multiplication
+   */
+  public int multiply(int x, int y) {
+    assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize());
+    return mulTable[x][y];
+  }
+
+  /**
+   * Compute the division of two fields.
+   *
+   * @param x input field
+   * @param y input field
+   * @return x/y
+   */
+  public int divide(int x, int y) {
+    assert (x >= 0 && x < getFieldSize() && y > 0 && y < getFieldSize());
+    return divTable[x][y];
+  }
+
+  /**
+   * Compute power n of a field.
+   *
+   * @param x input field
+   * @param n power
+   * @return x^n
+   */
+  public int power(int x, int n) {
+    assert (x >= 0 && x < getFieldSize());
+    if (n == 0) {
+      return 1;
+    }
+    if (x == 0) {
+      return 0;
+    }
+    x = logTable[x] * n;
+    if (x < primitivePeriod) {
+      return powTable[x];
+    }
+    x = x % primitivePeriod;
+    return powTable[x];
+  }
+
+  /**
+   * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such
+   * that Vz=y. The output z will be placed in y.
+   *
+   * @param x the vector which describe the Vandermonde matrix
+   * @param y right-hand side of the Vandermonde system equation. will be
+   *          replaced the output in this vector
+   */
+  public void solveVandermondeSystem(int[] x, int[] y) {
+    solveVandermondeSystem(x, y, x.length);
+  }
+
+  /**
+   * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such
+   * that Vz=y. The output z will be placed in y.
+   *
+   * @param x   the vector which describe the Vandermonde matrix
+   * @param y   right-hand side of the Vandermonde system equation. will be
+   *            replaced the output in this vector
+   * @param len consider x and y only from 0...len-1
+   */
+  public void solveVandermondeSystem(int[] x, int[] y, int len) {
+    assert (x.length <= len && y.length <= len);
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        y[j] = y[j] ^ mulTable[x[i]][y[j - 1]];
+      }
+    }
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        y[j] = divTable[y[j]][x[j] ^ x[j - i - 1]];
+      }
+      for (int j = i; j < len - 1; j++) {
+        y[j] = y[j] ^ y[j + 1];
+      }
+    }
+  }
+
+  /**
+   * A "bulk" version to the solving of Vandermonde System.
+   */
+  public void solveVandermondeSystem(int[] x, byte[][] y, int[] outputOffsets,
+                                     int len, int dataLen) {
+    int idx1, idx2;
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        for (idx2 = outputOffsets[j - 1], idx1 =
+            outputOffsets[j]; idx1 < outputOffsets[j] + dataLen;
+            idx1++, idx2++) {
+          y[j][idx1] = (byte) (y[j][idx1] ^ mulTable[x[i]][y[j - 1][idx2] &
+              0x000000FF]);
+        }
+      }
+    }
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        for (idx1 = outputOffsets[j];
+             idx1 < outputOffsets[j] + dataLen; idx1++) {
+          y[j][idx1] = (byte) (divTable[y[j][idx1] & 0x000000FF][x[j] ^
+              x[j - i - 1]]);
+        }
+      }
+      for (int j = i; j < len - 1; j++) {
+        for (idx2 = outputOffsets[j + 1], idx1 = outputOffsets[j];
+             idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) {
+          y[j][idx1] = (byte) (y[j][idx1] ^ y[j + 1][idx2]);
+        }
+      }
+    }
+  }
+
+  /**
+   * A "bulk" version of the solveVandermondeSystem, using ByteBuffer.
+   */
+  public void solveVandermondeSystem(int[] x, ByteBuffer[] y, int len) {
+    ByteBuffer p;
+    int idx1, idx2;
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        p = y[j];
+        for (idx1 = p.position(), idx2 = y[j - 1].position(); idx1 < p
+            .limit(); idx1++, idx2++) {
+          p.put(idx1, (byte) (p.get(idx1) ^ mulTable[x[i]][y[j - 1].get(idx2) &
+              0x000000FF]));
+        }
+      }
+    }
+
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        p = y[j];
+        for (idx1 = p.position(); idx1 < p.limit(); idx1++) {
+          p.put(idx1, (byte) (divTable[p.get(idx1) &
+              0x000000FF][x[j] ^ x[j - i - 1]]));
+        }
+      }
+
+      for (int j = i; j < len - 1; j++) {
+        p = y[j];
+        for (idx1 = p.position(), idx2 = y[j + 1].position(); idx1 < p
+            .limit(); idx1++, idx2++) {
+          p.put(idx1, (byte) (p.get(idx1) ^ y[j + 1].get(idx2)));
+        }
+      }
+    }
+  }
+
+  /**
+   * Compute the multiplication of two polynomials. The index in the array
+   * corresponds to the power of the entry. For example p[0] is the constant
+   * term of the polynomial p.
+   *
+   * @param p input polynomial
+   * @param q input polynomial
+   * @return polynomial represents p*q
+   */
+  public int[] multiply(int[] p, int[] q) {
+    int len = p.length + q.length - 1;
+    int[] result = new int[len];
+    for (int i = 0; i < len; i++) {
+      result[i] = 0;
+    }
+    for (int i = 0; i < p.length; i++) {
+
+      for (int j = 0; j < q.length; j++) {
+        result[i + j] = add(result[i + j], multiply(p[i], q[j]));
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Compute the remainder of a dividend and divisor pair. The index in the
+   * array corresponds to the power of the entry. For example p[0] is the
+   * constant term of the polynomial p.
+   *
+   * @param dividend dividend polynomial, the remainder will be placed
+   *                 here when return
+   * @param divisor  divisor polynomial
+   */
+  public void remainder(int[] dividend, int[] divisor) {
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      int ratio = divTable[dividend[i +
+          divisor.length - 1]][divisor[divisor.length - 1]];
+      for (int j = 0; j < divisor.length; j++) {
+        int k = j + i;
+        dividend[k] = dividend[k] ^ mulTable[ratio][divisor[j]];
+      }
+    }
+  }
+
+  /**
+   * Compute the sum of two polynomials. The index in the array corresponds to
+   * the power of the entry. For example p[0] is the constant term of the
+   * polynomial p.
+   *
+   * @param p input polynomial
+   * @param q input polynomial
+   * @return polynomial represents p+q
+   */
+  public int[] add(int[] p, int[] q) {
+    int len = Math.max(p.length, q.length);
+    int[] result = new int[len];
+    for (int i = 0; i < len; i++) {
+      if (i < p.length && i < q.length) {
+        result[i] = add(p[i], q[i]);
+      } else if (i < p.length) {
+        result[i] = p[i];
+      } else {
+        result[i] = q[i];
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Substitute x into polynomial p(x).
+   *
+   * @param p input polynomial
+   * @param x input field
+   * @return p(x)
+   */
+  public int substitute(int[] p, int x) {
+    int result = 0;
+    int y = 1;
+    for (int i = 0; i < p.length; i++) {
+      result = result ^ mulTable[p[i]][y];
+      y = mulTable[x][y];
+    }
+    return result;
+  }
+
+  /**
+   * A "bulk" version of the substitute.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param q store the return result
+   * @param x input field
+   */
+  public void substitute(byte[][] p, byte[] q, int x) {
+    int y = 1;
+    for (int i = 0; i < p.length; i++) {
+      byte[] pi = p[i];
+      for (int j = 0; j < pi.length; j++) {
+        int pij = pi[j] & 0x000000FF;
+        q[j] = (byte) (q[j] ^ mulTable[pij][y]);
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
+   * A "bulk" version of the substitute.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param offsets
+   * @param len
+   * @param q store the return result
+   * @param offset
+   * @param x input field
+   */
+  public void substitute(byte[][] p, int[] offsets,
+                         int len, byte[] q, int offset, int x) {
+    int y = 1, iIdx, oIdx;
+    for (int i = 0; i < p.length; i++) {
+      byte[] pi = p[i];
+      for (iIdx = offsets[i], oIdx = offset;
+           iIdx < offsets[i] + len; iIdx++, oIdx++) {
+        int pij = pi != null ? pi[iIdx] & 0x000000FF : 0;
+        q[oIdx] = (byte) (q[oIdx] ^ mulTable[pij][y]);
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
+   * A "bulk" version of the substitute, using ByteBuffer.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param q store the return result
+   * @param x input field
+   */
+  public void substitute(ByteBuffer[] p, int len, ByteBuffer q, int x) {
+    int y = 1, iIdx, oIdx;
+    for (int i = 0; i < p.length; i++) {
+      ByteBuffer pi = p[i];
+      int pos = pi != null ? pi.position() : 0;
+      int limit = pi != null ? pi.limit() : len;
+      for (oIdx = q.position(), iIdx = pos;
+           iIdx < limit; iIdx++, oIdx++) {
+        int pij = pi != null ? pi.get(iIdx) & 0x000000FF : 0;
+        q.put(oIdx, (byte) (q.get(oIdx) ^ mulTable[pij][y]));
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
+   * The "bulk" version of the remainder.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(byte[][] dividend, int[] divisor) {
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      for (int j = 0; j < divisor.length; j++) {
+        for (int k = 0; k < dividend[i].length; k++) {
+          int ratio = divTable[dividend[i + divisor.length - 1][k] &
+              0x00FF][divisor[divisor.length - 1]];
+          dividend[j + i][k] = (byte) ((dividend[j + i][k] & 0x00FF) ^
+              mulTable[ratio][divisor[j]]);
+        }
+      }
+    }
+  }
+
+  /**
+   * The "bulk" version of the remainder.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(byte[][] dividend, int[] offsets,
+                        int len, int[] divisor) {
+    int idx1, idx2;
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      for (int j = 0; j < divisor.length; j++) {
+        for (idx2 = offsets[j + i], idx1 = offsets[i + divisor.length - 1];
+             idx1 < offsets[i + divisor.length - 1] + len;
+             idx1++, idx2++) {
+          int ratio = divTable[dividend[i + divisor.length - 1][idx1] &
+              0x00FF][divisor[divisor.length - 1]];
+          dividend[j + i][idx2] = (byte) ((dividend[j + i][idx2] & 0x00FF) ^
+              mulTable[ratio][divisor[j]]);
+        }
+      }
+    }
+  }
+
+  /**
+   * The "bulk" version of the remainder, using ByteBuffer.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(ByteBuffer[] dividend, int[] divisor) {
+    int idx1, idx2;
+    ByteBuffer b1, b2;
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      for (int j = 0; j < divisor.length; j++) {
+        b1 = dividend[i + divisor.length - 1];
+        b2 = dividend[j + i];
+        for (idx1 = b1.position(), idx2 = b2.position();
+             idx1 < b1.limit(); idx1++, idx2++) {
+          int ratio = divTable[b1.get(idx1) &
+              0x00FF][divisor[divisor.length - 1]];
+          b2.put(idx2, (byte) ((b2.get(idx2) & 0x00FF) ^
+              mulTable[ratio][divisor[j]]));
+        }
+      }
+    }
+  }
+
+  /**
+   * Perform Gaussian elimination on the given matrix. This matrix has to be a
+   * fat matrix (number of rows &gt; number of columns).
+   */
+  public void gaussianElimination(int[][] matrix) {
+    assert (matrix != null && matrix.length > 0 && matrix[0].length > 0
+        && matrix.length < matrix[0].length);
+    int height = matrix.length;
+    int width = matrix[0].length;
+    for (int i = 0; i < height; i++) {
+      boolean pivotFound = false;
+      // scan the column for a nonzero pivot and swap it to the diagonal
+      for (int j = i; j < height; j++) {
+        if (matrix[i][j] != 0) {
+          int[] tmp = matrix[i];
+          matrix[i] = matrix[j];
+          matrix[j] = tmp;
+          pivotFound = true;
+          break;
+        }
+      }
+      if (!pivotFound) {
+        continue;
+      }
+      int pivot = matrix[i][i];
+      for (int j = i; j < width; j++) {
+        matrix[i][j] = divide(matrix[i][j], pivot);
+      }
+      for (int j = i + 1; j < height; j++) {
+        int lead = matrix[j][i];
+        for (int k = i; k < width; k++) {
+          matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k]));
+        }
+      }
+    }
+    for (int i = height - 1; i >= 0; i--) {
+      for (int j = 0; j < i; j++) {
+        int lead = matrix[j][i];
+        for (int k = i; k < width; k++) {
+          matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k]));
+        }
+      }
+    }
+  }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/RSUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/RSUtil.java
new file mode 100644
index 0000000..0ec88d0
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/RSUtil.java
@@ -0,0 +1,188 @@
+/**
+ * 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.ozone.erasurecode.rawcoder.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Utilities for implementing Reed-Solomon code, used by RS coder. Some of the
+ * codes are borrowed from ISA-L implementation (C or ASM codes).
+ */
+@InterfaceAudience.Private
+public final class RSUtil {
+
+  private RSUtil() {
+  }
+
+  // We always use the byte system (with symbol size 8, field size 256,
+  // primitive polynomial 285, and primitive root 2).
+  public static final GaloisField GF = GaloisField.getInstance();
+  public static final int PRIMITIVE_ROOT = 2;
+
+  public static int[] getPrimitivePower(int numDataUnits, int numParityUnits) {
+    int[] primitivePower = new int[numDataUnits + numParityUnits];
+    // compute powers of the primitive root
+    for (int i = 0; i < numDataUnits + numParityUnits; i++) {
+      primitivePower[i] = GF.power(PRIMITIVE_ROOT, i);
+    }
+    return primitivePower;
+  }
+
+  public static void initTables(int k, int rows, byte[] codingMatrix,
+      int matrixOffset, byte[] gfTables) {
+    int i, j;
+
+    int offset = 0, idx = matrixOffset;
+    for (i = 0; i < rows; i++) {
+      for (j = 0; j < k; j++) {
+        GF256.gfVectMulInit(codingMatrix[idx++], gfTables, offset);
+        offset += 32;
+      }
+    }
+  }
+
+  /**
+   * Ported from Intel ISA-L library.
+   */
+  public static void genCauchyMatrix(byte[] a, int m, int k) {
+    // Identity matrix in high position
+    for (int i = 0; i < k; i++) {
+      a[k * i + i] = 1;
+    }
+
+    // For the rest choose 1/(i + j) | i != j
+    int pos = k * k;
+    for (int i = k; i < m; i++) {
+      for (int j = 0; j < k; j++) {
+        a[pos++] = GF256.gfInv((byte) (i ^ j));
+      }
+    }
+  }
+
+  /**
+   * Encode a group of inputs data and generate the outputs. It's also used for
+   * decoding because, in this implementation, encoding and decoding are
+   * unified.
+   *
+   * The algorithm is ported from Intel ISA-L library for compatible. It
+   * leverages Java auto-vectorization support for performance.
+   */
+  public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs,
+      int[] inputOffsets, byte[][] outputs,
+      int[] outputOffsets) {
+    int numInputs = inputs.length;
+    int numOutputs = outputs.length;
+    int l, i, j, iPos, oPos;
+    byte[] input, output;
+    byte s;
+    final int times = dataLen / 8;
+    final int extra = dataLen - dataLen % 8;
+    byte[] tableLine;
+
+    for (l = 0; l < numOutputs; l++) {
+      output = outputs[l];
+
+      for (j = 0; j < numInputs; j++) {
+        input = inputs[j];
+        iPos = inputOffsets[j];
+        oPos = outputOffsets[l];
+
+        s = gfTables[j * 32 + l * numInputs * 32 + 1];
+        tableLine = GF256.gfMulTab()[s & 0xff];
+
+        /**
+         * Purely for performance, assuming we can use 8 bytes in the SIMD
+         * instruction. Subject to be improved.
+         */
+        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+          output[oPos + 0] ^= tableLine[0xff & input[iPos + 0]];
+          output[oPos + 1] ^= tableLine[0xff & input[iPos + 1]];
+          output[oPos + 2] ^= tableLine[0xff & input[iPos + 2]];
+          output[oPos + 3] ^= tableLine[0xff & input[iPos + 3]];
+          output[oPos + 4] ^= tableLine[0xff & input[iPos + 4]];
+          output[oPos + 5] ^= tableLine[0xff & input[iPos + 5]];
+          output[oPos + 6] ^= tableLine[0xff & input[iPos + 6]];
+          output[oPos + 7] ^= tableLine[0xff & input[iPos + 7]];
+        }
+
+        /**
+         * For the left bytes, do it one by one.
+         */
+        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+          output[oPos] ^= tableLine[0xff & input[iPos]];
+        }
+      }
+    }
+  }
+
+  /**
+   * See above. Try to use the byte[] version when possible.
+   */
+  public static void encodeData(byte[] gfTables, ByteBuffer[] inputs,
+      ByteBuffer[] outputs) {
+    int numInputs = inputs.length;
+    int numOutputs = outputs.length;
+    int dataLen = inputs[0].remaining();
+    int l, i, j, iPos, oPos;
+    ByteBuffer input, output;
+    byte s;
+    final int times = dataLen / 8;
+    final int extra = dataLen - dataLen % 8;
+    byte[] tableLine;
+
+    for (l = 0; l < numOutputs; l++) {
+      output = outputs[l];
+
+      for (j = 0; j < numInputs; j++) {
+        input = inputs[j];
+        iPos = input.position();
+        oPos = output.position();
+
+        s = gfTables[j * 32 + l * numInputs * 32 + 1];
+        tableLine = GF256.gfMulTab()[s & 0xff];
+
+        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+          output.put(oPos + 0, (byte) (output.get(oPos + 0) ^
+              tableLine[0xff & input.get(iPos + 0)]));
+          output.put(oPos + 1, (byte) (output.get(oPos + 1) ^
+              tableLine[0xff & input.get(iPos + 1)]));
+          output.put(oPos + 2, (byte) (output.get(oPos + 2) ^
+              tableLine[0xff & input.get(iPos + 2)]));
+          output.put(oPos + 3, (byte) (output.get(oPos + 3) ^
+              tableLine[0xff & input.get(iPos + 3)]));
+          output.put(oPos + 4, (byte) (output.get(oPos + 4) ^
+              tableLine[0xff & input.get(iPos + 4)]));
+          output.put(oPos + 5, (byte) (output.get(oPos + 5) ^
+              tableLine[0xff & input.get(iPos + 5)]));
+          output.put(oPos + 6, (byte) (output.get(oPos + 6) ^
+              tableLine[0xff & input.get(iPos + 6)]));
+          output.put(oPos + 7, (byte) (output.get(oPos + 7) ^
+              tableLine[0xff & input.get(iPos + 7)]));
+        }
+
+        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+          output.put(oPos, (byte) (output.get(oPos) ^
+              tableLine[0xff & input.get(iPos)]));
+        }
+      }
+    }
+  }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/package-info.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/package-info.java
new file mode 100644
index 0000000..5e6fac4
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/package-info.java
@@ -0,0 +1,27 @@
+/**
+ * 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.
+ */
+
+/**
+ * General helpers for implementing raw erasure coders.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.ozone.erasurecode.rawcoder.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/resources/META-INF/services/org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory b/hadoop-hdds/erasurecode/src/main/resources/META-INF/services/org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory
new file mode 100644
index 0000000..24625b5
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/resources/META-INF/services/org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory
@@ -0,0 +1,17 @@
+#
+#   Licensed 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.
+#
+org.apache.ozone.erasurecode.rawcoder.RSRawErasureCoderFactory
+org.apache.ozone.erasurecode.rawcoder.XORRawErasureCoderFactory
+org.apache.ozone.erasurecode.rawcoder.NativeRSRawErasureCoderFactory
+org.apache.ozone.erasurecode.rawcoder.NativeXORRawErasureCoderFactory
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/DumpUtil.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/DumpUtil.java
new file mode 100644
index 0000000..fbbf4f6
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/DumpUtil.java
@@ -0,0 +1,96 @@
+/**
+ * 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.ozone.erasurecode;
+
+
+/**
+ * A dump utility class for debugging data erasure coding/decoding issues.
+ * Don't suggest they are used in runtime production codes.
+ */
+public final class DumpUtil {
+  private static final String HEX_CHARS_STR = "0123456789ABCDEF";
+  private static final char[] HEX_CHARS = HEX_CHARS_STR.toCharArray();
+
+  private DumpUtil() {
+    // No called
+  }
+
+  /**
+   * Convert bytes into format like 0x02 02 00 80.
+   * If limit is negative or too large, then all bytes will be converted.
+   */
+  public static String bytesToHex(byte[] bytes, int limit) {
+    if (limit <= 0 || limit > bytes.length) {
+      limit = bytes.length;
+    }
+    int len = limit * 2;
+    len += limit; // for ' ' appended for each char
+    len += 2; // for '0x' prefix
+    char[] hexChars = new char[len];
+    hexChars[0] = '0';
+    hexChars[1] = 'x';
+    for (int j = 0; j < limit; j++) {
+      int v = bytes[j] & 0xFF;
+      hexChars[j * 3 + 2] = HEX_CHARS[v >>> 4];
+      hexChars[j * 3 + 3] = HEX_CHARS[v & 0x0F];
+      hexChars[j * 3 + 4] = ' ';
+    }
+
+    return new String(hexChars);
+  }
+
+  public static void dumpMatrix(byte[] matrix,
+                                int numDataUnits, int numAllUnits) {
+    for (int i = 0; i < numDataUnits; i++) {
+      for (int j = 0; j < numAllUnits; j++) {
+        System.out.print(" ");
+        System.out.print(0xff & matrix[i * numAllUnits + j]);
+      }
+      System.out.println();
+    }
+  }
+
+  /**
+   * Print data in hex format in an array of chunks.
+   * @param header
+   * @param chunks
+   */
+  public static void dumpChunks(String header, ECChunk[] chunks) {
+    System.out.println();
+    System.out.println(header);
+    for (int i = 0; i < chunks.length; i++) {
+      dumpChunk(chunks[i]);
+    }
+    System.out.println();
+  }
+
+  /**
+   * Print data in hex format in a chunk.
+   * @param chunk
+   */
+  public static void dumpChunk(ECChunk chunk) {
+    String str;
+    if (chunk == null) {
+      str = "<EMPTY>";
+    } else {
+      byte[] bytes = chunk.toBytesArray();
+      str = DumpUtil.bytesToHex(bytes, 16);
+    }
+    System.out.println(str);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCodecRegistry.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCodecRegistry.java
new file mode 100644
index 0000000..efc3a9f
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCodecRegistry.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.NativeXORRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RSRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.ozone.erasurecode.rawcoder.XORRawErasureCoderFactory;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test CodecRegistry.
+ */
+public class TestCodecRegistry {
+
+  @Test
+  public void testGetCodecs() {
+    Set<String> codecs = CodecRegistry.getInstance().getCodecNames();
+    assertEquals(2, codecs.size());
+    assertTrue(
+        codecs.contains(ECReplicationConfig.EcCodec.RS.name().toLowerCase()));
+    assertTrue(
+        codecs.contains(ECReplicationConfig.EcCodec.XOR.name().toLowerCase()));
+  }
+
+  @Test
+  public void testGetCoders() {
+    List<RawErasureCoderFactory> coders = CodecRegistry.getInstance().
+        getCoders(ECReplicationConfig.EcCodec.RS.name().toLowerCase());
+    assertEquals(2, coders.size());
+    assertTrue(coders.get(0) instanceof NativeRSRawErasureCoderFactory);
+    assertTrue(coders.get(1) instanceof RSRawErasureCoderFactory);
+
+    coders = CodecRegistry.getInstance().
+        getCoders(ECReplicationConfig.EcCodec.XOR.name().toLowerCase());
+    assertEquals(2, coders.size());
+    assertTrue(coders.get(0) instanceof NativeXORRawErasureCoderFactory);
+    assertTrue(coders.get(1) instanceof XORRawErasureCoderFactory);
+  }
+
+  @Test
+  public void testGetCodersWrong() {
+    List<RawErasureCoderFactory> coders =
+        CodecRegistry.getInstance().getCoders("WRONG_CODEC");
+    assertNull(coders);
+  }
+
+  @Test
+  public void testGetCoderByNameWrong() {
+    RawErasureCoderFactory coder = CodecRegistry.getInstance().
+        getCoderByName(ECReplicationConfig.EcCodec.RS.name().toLowerCase(),
+            "WRONG_RS");
+    assertNull(coder);
+  }
+
+  @Test
+  public void testUpdateCoders() {
+    class RSUserDefinedIncorrectFactory implements RawErasureCoderFactory {
+      public RawErasureEncoder createEncoder(ECReplicationConfig coderOptions) {
+        return null;
+      }
+
+      public RawErasureDecoder createDecoder(ECReplicationConfig coderOptions) {
+        return null;
+      }
+
+      public String getCoderName() {
+        return "rs_java";
+      }
+
+      public String getCodecName() {
+        return ECReplicationConfig.EcCodec.RS.name().toLowerCase();
+      }
+    }
+
+    List<RawErasureCoderFactory> userDefinedFactories = new ArrayList<>();
+    userDefinedFactories.add(new RSUserDefinedIncorrectFactory());
+    CodecRegistry.getInstance().updateCoders(userDefinedFactories);
+
+    // check RS coders
+    List<RawErasureCoderFactory> rsCoders = CodecRegistry.getInstance().
+        getCoders(ECReplicationConfig.EcCodec.RS.name().toLowerCase());
+    assertEquals(2, rsCoders.size());
+    assertTrue(rsCoders.get(0) instanceof NativeRSRawErasureCoderFactory);
+    assertTrue(rsCoders.get(1) instanceof RSRawErasureCoderFactory);
+
+    // check RS coder names
+    String[] rsCoderNames = CodecRegistry.getInstance().
+        getCoderNames(ECReplicationConfig.EcCodec.RS.name().toLowerCase());
+    assertEquals(2, rsCoderNames.length);
+    assertEquals(NativeRSRawErasureCoderFactory.CODER_NAME, rsCoderNames[0]);
+    assertEquals(RSRawErasureCoderFactory.CODER_NAME, rsCoderNames[1]);
+  }
+
+  @Test
+  public void testGetCoderNames() {
+    String[] coderNames = CodecRegistry.getInstance().
+        getCoderNames(ECReplicationConfig.EcCodec.RS.name().toLowerCase());
+    assertEquals(2, coderNames.length);
+    assertEquals(NativeRSRawErasureCoderFactory.CODER_NAME, coderNames[0]);
+    assertEquals(RSRawErasureCoderFactory.CODER_NAME, coderNames[1]);
+
+    coderNames = CodecRegistry.getInstance().
+        getCoderNames(ECReplicationConfig.EcCodec.XOR.name().toLowerCase());
+    assertEquals(2, coderNames.length);
+    assertEquals(NativeXORRawErasureCoderFactory.CODER_NAME, coderNames[0]);
+    assertEquals(XORRawErasureCoderFactory.CODER_NAME, coderNames[1]);
+  }
+
+  @Test
+  public void testGetCoderByName() {
+    RawErasureCoderFactory coder = CodecRegistry.getInstance().
+        getCoderByName(ECReplicationConfig.EcCodec.RS.name().toLowerCase(),
+            RSRawErasureCoderFactory.CODER_NAME);
+    assertTrue(coder instanceof RSRawErasureCoderFactory);
+
+    coder = CodecRegistry.getInstance()
+        .getCoderByName(ECReplicationConfig.EcCodec.RS.name().toLowerCase(),
+            NativeRSRawErasureCoderFactory.CODER_NAME);
+    assertTrue(coder instanceof NativeRSRawErasureCoderFactory);
+
+    coder = CodecRegistry.getInstance()
+        .getCoderByName(ECReplicationConfig.EcCodec.XOR.name().toLowerCase(),
+            XORRawErasureCoderFactory.CODER_NAME);
+    assertTrue(coder instanceof XORRawErasureCoderFactory);
+
+    coder = CodecRegistry.getInstance()
+        .getCoderByName(ECReplicationConfig.EcCodec.XOR.name().toLowerCase(),
+            NativeXORRawErasureCoderFactory.CODER_NAME);
+    assertTrue(coder instanceof NativeXORRawErasureCoderFactory);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCoderBase.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCoderBase.java
new file mode 100644
index 0000000..e4b02cd
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCoderBase.java
@@ -0,0 +1,524 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test base of common utilities for tests not only raw coders but also block
+ * coders.
+ */
+@SuppressWarnings({"checkstyle:VisibilityModifier", "checkstyle:HiddenField"})
+public abstract class TestCoderBase {
+  protected static final Random RAND = new Random();
+  private static int fixedDataGenerator = 0;
+  protected boolean allowDump = true;
+  protected int numDataUnits;
+  protected int numParityUnits;
+  protected int baseChunkSize = 1024;
+  // Indexes of erased data units.
+  protected int[] erasedDataIndexes = new int[]{0};
+  // Indexes of erased parity units.
+  protected int[] erasedParityIndexes = new int[]{0};
+  // Data buffers are either direct or on-heap, for performance the two cases
+  // may go to different coding implementations.
+  protected boolean usingDirectBuffer = true;
+  protected boolean usingFixedData = true;
+  protected byte[][] fixedData;
+  private int chunkSize = baseChunkSize;
+  private BufferAllocator allocator;
+  private byte[] zeroChunkBytes;
+  private boolean startBufferWithZero = true;
+  // Using this the generated data can be repeatable across multiple calls to
+  // encode(), in order for troubleshooting.
+  private ConfigurationSource conf;
+
+  protected int getChunkSize() {
+    return chunkSize;
+  }
+
+  protected void setChunkSize(int chunkSize) {
+    this.chunkSize = chunkSize;
+    this.zeroChunkBytes = new byte[chunkSize]; // With ZERO by default
+  }
+
+  protected byte[] getZeroChunkBytes() {
+    return zeroChunkBytes;
+  }
+
+  protected void prepareBufferAllocator(boolean usingSlicedBuffer) {
+    if (usingSlicedBuffer) {
+      int roughEstimationSpace =
+          chunkSize * (numDataUnits + numParityUnits) * 10;
+      allocator = new BufferAllocator.SlicedBufferAllocator(usingDirectBuffer,
+          roughEstimationSpace);
+    } else {
+      allocator = new BufferAllocator.SimpleBufferAllocator(usingDirectBuffer);
+    }
+  }
+
+  protected boolean isAllowDump() {
+    return allowDump;
+  }
+
+  /**
+   * Prepare before running the case.
+   *
+   * @param usingFixedData Using fixed or pre-generated data to test instead of
+   *                       generating data
+   */
+  protected void prepare(ConfigurationSource conf,
+      int numDataUnits,
+      int numParityUnits,
+      int[] erasedDataIndexes,
+      int[] erasedParityIndexes,
+      boolean usingFixedData) {
+    this.conf = conf != null ? conf : new OzoneConfiguration();
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+    this.erasedDataIndexes = erasedDataIndexes != null ?
+        erasedDataIndexes : new int[]{0};
+    this.erasedParityIndexes = erasedParityIndexes != null ?
+        erasedParityIndexes : new int[]{0};
+    this.usingFixedData = usingFixedData;
+    if (usingFixedData) {
+      prepareFixedData();
+    }
+  }
+
+  /**
+   * Prepare before running the case.
+   *
+   * @param conf
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedDataIndexes
+   * @param erasedParityIndexes
+   */
+  protected void prepare(ConfigurationSource conf, int numDataUnits,
+      int numParityUnits, int[] erasedDataIndexes,
+      int[] erasedParityIndexes) {
+    prepare(conf, numDataUnits, numParityUnits, erasedDataIndexes,
+        erasedParityIndexes, false);
+  }
+
+  /**
+   * Prepare before running the case.
+   *
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedDataIndexes
+   * @param erasedParityIndexes
+   */
+  protected void prepare(
+      int numDataUnits,
+      int numParityUnits,
+      int[] erasedDataIndexes,
+      int[] erasedParityIndexes) {
+    prepare(null, numDataUnits, numParityUnits, erasedDataIndexes,
+        erasedParityIndexes, false);
+  }
+
+  /**
+   * Get the conf the test.
+   *
+   * @return configuration
+   */
+  protected ConfigurationSource getConf() {
+    return this.conf;
+  }
+
+  /**
+   * Compare and verify if erased chunks are equal to recovered chunks.
+   *
+   * @param erasedChunks
+   * @param recoveredChunks
+   */
+  protected void compareAndVerify(ECChunk[] erasedChunks,
+      ECChunk[] recoveredChunks) {
+    byte[][] erased = toArrays(erasedChunks);
+    byte[][] recovered = toArrays(recoveredChunks);
+    boolean result = Arrays.deepEquals(erased, recovered);
+    if (!result) {
+      assertTrue("Decoding and comparing failed.", result);
+    }
+  }
+
+  /**
+   * Adjust and return erased indexes altogether, including erased data indexes
+   * and parity indexes.
+   * @return erased indexes altogether
+   */
+  protected int[] getErasedIndexesForDecoding() {
+    int[] erasedIndexesForDecoding =
+        new int[erasedDataIndexes.length + erasedParityIndexes.length];
+
+    int idx = 0;
+
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
+      erasedIndexesForDecoding[idx++] = erasedDataIndexes[i];
+    }
+
+    for (int i = 0; i < erasedParityIndexes.length; i++) {
+      erasedIndexesForDecoding[idx++] = erasedParityIndexes[i] + numDataUnits;
+    }
+
+    return erasedIndexesForDecoding;
+  }
+
+  /**
+   * Return input chunks for decoding, which is dataChunks + parityChunks.
+   *
+   * @param dataChunks
+   * @param parityChunks
+   * @return
+   */
+  protected ECChunk[] prepareInputChunksForDecoding(ECChunk[] dataChunks,
+      ECChunk[] parityChunks) {
+    ECChunk[] inputChunks = new ECChunk[numDataUnits + numParityUnits];
+
+    int idx = 0;
+
+    for (int i = 0; i < numDataUnits; i++) {
+      inputChunks[idx++] = dataChunks[i];
+    }
+
+    for (int i = 0; i < numParityUnits; i++) {
+      inputChunks[idx++] = parityChunks[i];
+    }
+
+    return inputChunks;
+  }
+
+  /**
+   * Erase some data chunks to test the recovering of them. As they're erased,
+   * we don't need to read them and will not have the buffers at all, so just
+   * set them as null.
+   *
+   * @param dataChunks
+   * @param parityChunks
+   * @return clone of erased chunks
+   */
+  protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks,
+      ECChunk[] parityChunks) {
+    ECChunk[] toEraseChunks = new ECChunk[erasedDataIndexes.length +
+        erasedParityIndexes.length];
+
+    int idx = 0;
+
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
+      toEraseChunks[idx++] = dataChunks[erasedDataIndexes[i]];
+      dataChunks[erasedDataIndexes[i]] = null;
+    }
+
+    for (int i = 0; i < erasedParityIndexes.length; i++) {
+      toEraseChunks[idx++] = parityChunks[erasedParityIndexes[i]];
+      parityChunks[erasedParityIndexes[i]] = null;
+    }
+
+    return toEraseChunks;
+  }
+
+  /**
+   * Erase data from the specified chunks, just setting them as null.
+   * @param chunks
+   */
+  protected void eraseDataFromChunks(ECChunk[] chunks) {
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = null;
+    }
+  }
+
+  protected void markChunks(ECChunk[] chunks) {
+    for (int i = 0; i < chunks.length; i++) {
+      if (chunks[i] != null) {
+        chunks[i].getBuffer().mark();
+      }
+    }
+  }
+
+  protected void restoreChunksFromMark(ECChunk[] chunks) {
+    for (int i = 0; i < chunks.length; i++) {
+      if (chunks[i] != null) {
+        chunks[i].getBuffer().reset();
+      }
+    }
+  }
+
+  /**
+   * Clone chunks along with copying the associated data. It respects how the
+   * chunk buffer is allocated, direct or non-direct. It avoids affecting the
+   * original chunk buffers.
+   * @param chunks
+   * @return
+   */
+  protected ECChunk[] cloneChunksWithData(ECChunk[] chunks) {
+    ECChunk[] results = new ECChunk[chunks.length];
+    for (int i = 0; i < chunks.length; i++) {
+      results[i] = cloneChunkWithData(chunks[i]);
+    }
+
+    return results;
+  }
+
+  /**
+   * Clone chunk along with copying the associated data. It respects how the
+   * chunk buffer is allocated, direct or non-direct. It avoids affecting the
+   * original chunk.
+   * @param chunk
+   * @return a new chunk
+   */
+  protected ECChunk cloneChunkWithData(ECChunk chunk) {
+    if (chunk == null) {
+      return null;
+    }
+
+    ByteBuffer srcBuffer = chunk.getBuffer();
+
+    byte[] bytesArr = new byte[srcBuffer.remaining()];
+    srcBuffer.mark();
+    srcBuffer.get(bytesArr, 0, bytesArr.length);
+    srcBuffer.reset();
+
+    ByteBuffer destBuffer = allocateOutputBuffer(bytesArr.length);
+    int pos = destBuffer.position();
+    destBuffer.put(bytesArr);
+    destBuffer.flip();
+    destBuffer.position(pos);
+
+    return new ECChunk(destBuffer);
+  }
+
+  /**
+   * Allocate a chunk for output or writing.
+   * @return
+   */
+  protected ECChunk allocateOutputChunk() {
+    ByteBuffer buffer = allocateOutputBuffer(chunkSize);
+
+    return new ECChunk(buffer);
+  }
+
+  /**
+   * Allocate a buffer for output or writing. It can prepare for two kinds of
+   * data buffers: one with position as 0, the other with position > 0
+   * @return a buffer ready to write chunkSize bytes from current position
+   */
+  protected ByteBuffer allocateOutputBuffer(int bufferLen) {
+    /**
+     * When startBufferWithZero, will prepare a buffer as:---------------
+     * otherwise, the buffer will be like:             ___TO--BE--WRITTEN___,
+     * and in the beginning, dummy data are prefixed, to simulate a buffer of
+     * position > 0.
+     */
+    int startOffset = startBufferWithZero ? 0 : 11; // 11 is arbitrary
+    int allocLen = startOffset + bufferLen + startOffset;
+    ByteBuffer buffer = allocator.allocate(allocLen);
+    buffer.limit(startOffset + bufferLen);
+    fillDummyData(buffer, startOffset);
+    startBufferWithZero = !startBufferWithZero;
+
+    return buffer;
+  }
+
+  /**
+   * Prepare data chunks for each data unit, by generating random data.
+   * @return
+   */
+  protected ECChunk[] prepareDataChunksForEncoding() {
+    if (usingFixedData) {
+      ECChunk[] chunks = new ECChunk[numDataUnits];
+      for (int i = 0; i < chunks.length; i++) {
+        chunks[i] = makeChunkUsingData(fixedData[i]);
+      }
+      return chunks;
+    }
+
+    return generateDataChunks();
+  }
+
+  private ECChunk makeChunkUsingData(byte[] data) {
+    ECChunk chunk = allocateOutputChunk();
+    ByteBuffer buffer = chunk.getBuffer();
+    int pos = buffer.position();
+    buffer.put(data, 0, chunkSize);
+    buffer.flip();
+    buffer.position(pos);
+
+    return chunk;
+  }
+
+  private ECChunk[] generateDataChunks() {
+    ECChunk[] chunks = new ECChunk[numDataUnits];
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = generateDataChunk();
+    }
+
+    return chunks;
+  }
+
+  private void prepareFixedData() {
+    // We may load test data from a resource, or just generate randomly.
+    // The generated data will be used across subsequent encode/decode calls.
+    this.fixedData = new byte[numDataUnits][];
+    for (int i = 0; i < numDataUnits; i++) {
+      fixedData[i] = generateFixedData(baseChunkSize * 2);
+    }
+  }
+
+  /**
+   * Generate data chunk by making random data.
+   * @return
+   */
+  protected ECChunk generateDataChunk() {
+    ByteBuffer buffer = allocateOutputBuffer(chunkSize);
+    int pos = buffer.position();
+    buffer.put(generateData(chunkSize));
+    buffer.flip();
+    buffer.position(pos);
+
+    return new ECChunk(buffer);
+  }
+
+  /**
+   * Fill len of dummy data in the buffer at the current position.
+   * @param buffer
+   * @param len
+   */
+  protected void fillDummyData(ByteBuffer buffer, int len) {
+    byte[] dummy = new byte[len];
+    RAND.nextBytes(dummy);
+    buffer.put(dummy);
+  }
+
+  protected byte[] generateData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) RAND.nextInt(256);
+    }
+    return buffer;
+  }
+
+  protected byte[] generateFixedData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) fixedDataGenerator++;
+      if (fixedDataGenerator == 256) {
+        fixedDataGenerator = 0;
+      }
+    }
+    return buffer;
+  }
+
+  /**
+   * Prepare parity chunks for encoding, each chunk for each parity unit.
+   * @return
+   */
+  protected ECChunk[] prepareParityChunksForEncoding() {
+    ECChunk[] chunks = new ECChunk[numParityUnits];
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = allocateOutputChunk();
+    }
+
+    return chunks;
+  }
+
+  /**
+   * Prepare output chunks for decoding, each output chunk for each erased
+   * chunk.
+   * @return
+   */
+  protected ECChunk[] prepareOutputChunksForDecoding() {
+    ECChunk[] chunks = new ECChunk[erasedDataIndexes.length +
+        erasedParityIndexes.length];
+
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = allocateOutputChunk();
+    }
+
+    return chunks;
+  }
+
+  /**
+   * Convert an array of this chunks to an array of byte array.
+   * Note the chunk buffers are not affected.
+   * @param chunks
+   * @return an array of byte array
+   */
+  protected byte[][] toArrays(ECChunk[] chunks) {
+    byte[][] bytesArr = new byte[chunks.length][];
+
+    for (int i = 0; i < chunks.length; i++) {
+      if (chunks[i] != null) {
+        bytesArr[i] = chunks[i].toBytesArray();
+      }
+    }
+
+    return bytesArr;
+  }
+
+  /**
+   * Dump all the settings used in the test case if isAllowingVerboseDump
+   * is enabled.
+   */
+  protected void dumpSetting() {
+    if (allowDump) {
+      StringBuilder sb = new StringBuilder("Erasure coder test settings:\n");
+      sb.append(" numDataUnits=").append(numDataUnits);
+      sb.append(" numParityUnits=").append(numParityUnits);
+      sb.append(" chunkSize=").append(chunkSize).append("\n");
+
+      sb.append(" erasedDataIndexes=").
+          append(Arrays.toString(erasedDataIndexes));
+      sb.append(" erasedParityIndexes=").
+          append(Arrays.toString(erasedParityIndexes));
+      sb.append(" usingDirectBuffer=").append(usingDirectBuffer);
+      sb.append(" allowVerboseDump=").append(allowDump);
+      sb.append("\n");
+
+      System.out.println(sb.toString());
+    }
+  }
+
+  /**
+   * Dump chunks prefixed with a header if isAllowingVerboseDump is enabled.
+   */
+  protected void dumpChunks(String header, ECChunk[] chunks) {
+    if (allowDump) {
+      DumpUtil.dumpChunks(header, chunks);
+    }
+  }
+
+  /**
+   * Make some chunk messy or not correct any more.
+   */
+  protected void corruptSomeChunk(ECChunk[] chunks) {
+    int idx = new Random().nextInt(chunks.length);
+    ByteBuffer buffer = chunks[idx].getBuffer();
+    if (buffer.hasRemaining()) {
+      buffer.position(buffer.position() + 1);
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/package-info.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/package-info.java
new file mode 100644
index 0000000..c034c0b
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Unit tests for EC related classes.
+ */
+package org.apache.ozone.erasurecode;
+
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderBenchmark.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderBenchmark.java
new file mode 100644
index 0000000..326950d
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderBenchmark.java
@@ -0,0 +1,412 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.util.StopWatch;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A benchmark tool to test the performance of different erasure coders.
+ * The tool launches multiple threads to encode/decode certain amount of data,
+ * and measures the total throughput. It only focuses on performance and doesn't
+ * validate correctness of the encoded/decoded results.
+ * User can specify the data size each thread processes, as well as the chunk
+ * size to use for the coder.
+ * Different coders are supported. User can specify the coder by a coder index.
+ * The coder is shared among all the threads.
+ */
+public final class RawErasureCoderBenchmark {
+
+  private RawErasureCoderBenchmark() {
+    // prevent instantiation
+  }
+
+  // target size of input data buffer
+  private static final int TARGET_BUFFER_SIZE_MB = 126;
+
+  private static final int MAX_CHUNK_SIZE =
+      TARGET_BUFFER_SIZE_MB / BenchData.NUM_DATA_UNITS * 1024;
+
+  private static final List<RawErasureCoderFactory> CODER_MAKERS =
+      Collections.unmodifiableList(
+          Arrays.asList(new DummyRawErasureCoderFactory(),
+              new RSRawErasureCoderFactory()));
+
+  enum CODER {
+    DUMMY_CODER("Dummy coder"),
+    RS_CODER("Reed-Solomon Java coder");
+
+    private final String name;
+
+    CODER(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+  }
+
+  static {
+    Preconditions.checkArgument(CODER_MAKERS.size() == CODER.values().length);
+  }
+
+  private static void printAvailableCoders() {
+    StringBuilder sb = new StringBuilder(
+        "Available coders with coderIndex:\n");
+    for (CODER coder : CODER.values()) {
+      sb.append(coder.ordinal()).append(":").append(coder).append("\n");
+    }
+    System.out.println(sb);
+  }
+
+  private static void usage(String message) {
+    if (message != null) {
+      System.out.println(message);
+    }
+    System.out.println(
+        "Usage: RawErasureCoderBenchmark <encode/decode> <coderIndex> " +
+            "[numThreads] [dataSize-in-MB] [chunkSize-in-KB]");
+    printAvailableCoders();
+    System.exit(1);
+  }
+
+  public static void main(String[] args) throws Exception {
+    String opType = null;
+    int coderIndex = 0;
+    // default values
+    int dataSizeMB = 10240;
+    int chunkSizeKB = 1024;
+    int numThreads = 1;
+
+    if (args.length > 1) {
+      opType = args[0];
+      if (!"encode".equals(opType) && !"decode".equals(opType)) {
+        usage("Invalid type: should be either 'encode' or 'decode'");
+      }
+
+      try {
+        coderIndex = Integer.parseInt(args[1]);
+        if (coderIndex < 0 || coderIndex >= CODER.values().length) {
+          usage("Invalid coder index, should be [0-" +
+              (CODER.values().length - 1) + "]");
+        }
+      } catch (NumberFormatException e) {
+        usage("Malformed coder index, " + e.getMessage());
+      }
+    } else {
+      usage(null);
+    }
+
+    if (args.length > 2) {
+      try {
+        numThreads = Integer.parseInt(args[2]);
+        if (numThreads <= 0) {
+          usage("Invalid number of threads.");
+        }
+      } catch (NumberFormatException e) {
+        usage("Malformed number of threads, " + e.getMessage());
+      }
+    }
+
+    if (args.length > 3) {
+      try {
+        dataSizeMB = Integer.parseInt(args[3]);
+        if (dataSizeMB <= 0) {
+          usage("Invalid data size.");
+        }
+      } catch (NumberFormatException e) {
+        usage("Malformed data size, " + e.getMessage());
+      }
+    }
+
+    if (args.length > 4) {
+      try {
+        chunkSizeKB = Integer.parseInt(args[4]);
+        if (chunkSizeKB <= 0) {
+          usage("Chunk size should be positive.");
+        }
+        if (chunkSizeKB > MAX_CHUNK_SIZE) {
+          usage("Chunk size should be no larger than " + MAX_CHUNK_SIZE);
+        }
+      } catch (NumberFormatException e) {
+        usage("Malformed chunk size, " + e.getMessage());
+      }
+    }
+
+    performBench(opType, CODER.values()[coderIndex],
+        numThreads, dataSizeMB, chunkSizeKB);
+  }
+
+  /**
+   * Performs benchmark.
+   *
+   * @param opType      The operation to perform. Can be encode or decode
+   * @param coder       The coder to use
+   * @param numThreads  Number of threads to launch concurrently
+   * @param dataSizeMB  Total test data size in MB
+   * @param chunkSizeKB Chunk size in KB
+   */
+  public static void performBench(String opType, CODER coder,
+      int numThreads, int dataSizeMB, int chunkSizeKB) throws Exception {
+    BenchData.configure(dataSizeMB, chunkSizeKB);
+
+    RawErasureEncoder encoder = null;
+    RawErasureDecoder decoder = null;
+    ByteBuffer testData;
+    boolean isEncode = opType.equals("encode");
+
+    if (isEncode) {
+      encoder = getRawEncoder(coder.ordinal());
+      testData = genTestData(encoder.preferDirectBuffer(),
+          BenchData.bufferSizeKB);
+    } else {
+      decoder = getRawDecoder(coder.ordinal());
+      testData = genTestData(decoder.preferDirectBuffer(),
+          BenchData.bufferSizeKB);
+    }
+
+    ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+    List<Future<Long>> futures = new ArrayList<>(numThreads);
+    StopWatch sw = new StopWatch().start();
+    for (int i = 0; i < numThreads; i++) {
+      futures.add(executor.submit(new BenchmarkCallable(isEncode,
+          encoder, decoder, testData.duplicate())));
+    }
+    List<Long> durations = new ArrayList<>(numThreads);
+    try {
+      for (Future<Long> future : futures) {
+        durations.add(future.get());
+      }
+      long duration = sw.now(TimeUnit.MILLISECONDS);
+      double totalDataSize = BenchData.totalDataSizeKB * numThreads / 1024.0;
+      DecimalFormat df = new DecimalFormat("#.##");
+      System.out.println(coder + " " + opType + " " +
+          df.format(totalDataSize) + "MB data, with chunk size " +
+          BenchData.chunkSize / 1024 + "KB");
+      System.out.println("Total time: " + df.format(duration / 1000.0) + " s.");
+      System.out.println("Total throughput: " + df.format(
+          totalDataSize / duration * 1000.0) + " MB/s");
+      printThreadStatistics(durations, df);
+    } catch (Exception e) {
+      System.out.println("Error waiting for thread to finish.");
+      e.printStackTrace();
+      throw e;
+    } finally {
+      executor.shutdown();
+      if (encoder != null) {
+        encoder.release();
+      }
+      if (decoder != null) {
+        decoder.release();
+      }
+    }
+  }
+
+  private static RawErasureEncoder getRawEncoder(int index) throws IOException {
+    RawErasureEncoder encoder =
+        CODER_MAKERS.get(index).createEncoder(BenchData.OPTIONS);
+    final boolean isDirect = encoder.preferDirectBuffer();
+    encoder.encode(
+        getBufferForInit(BenchData.NUM_DATA_UNITS, 1, isDirect),
+        getBufferForInit(BenchData.NUM_PARITY_UNITS, 1, isDirect));
+    return encoder;
+  }
+
+  private static RawErasureDecoder getRawDecoder(int index) throws IOException {
+    RawErasureDecoder decoder =
+        CODER_MAKERS.get(index).createDecoder(BenchData.OPTIONS);
+    final boolean isDirect = decoder.preferDirectBuffer();
+    ByteBuffer[] inputs = getBufferForInit(
+        BenchData.NUM_ALL_UNITS, 1, isDirect);
+    for (int erasedIndex : BenchData.ERASED_INDEXES) {
+      inputs[erasedIndex] = null;
+    }
+    decoder.decode(inputs, BenchData.ERASED_INDEXES,
+        getBufferForInit(BenchData.ERASED_INDEXES.length, 1, isDirect));
+    return decoder;
+  }
+
+  private static ByteBuffer[] getBufferForInit(int numBuf,
+      int bufCap, boolean isDirect) {
+    ByteBuffer[] buffers = new ByteBuffer[numBuf];
+    for (int i = 0; i < buffers.length; i++) {
+      buffers[i] = isDirect ? ByteBuffer.allocateDirect(bufCap) :
+          ByteBuffer.allocate(bufCap);
+    }
+    return buffers;
+  }
+
+  private static void printThreadStatistics(
+      List<Long> durations, DecimalFormat df) {
+    Collections.sort(durations);
+    System.out.println("Threads statistics: ");
+    Double min = durations.get(0) / 1000.0;
+    Double max = durations.get(durations.size() - 1) / 1000.0;
+    Long sum = 0L;
+    for (Long duration : durations) {
+      sum += duration;
+    }
+    Double avg = sum.doubleValue() / durations.size() / 1000.0;
+    Double percentile = durations.get(
+        (int) Math.ceil(durations.size() * 0.9) - 1) / 1000.0;
+    System.out.println(durations.size() + " threads in total.");
+    System.out.println("Min: " + df.format(min) + " s, Max: " +
+        df.format(max) + " s, Avg: " + df.format(avg) +
+        " s, 90th Percentile: " + df.format(percentile) + " s.");
+  }
+
+  private static ByteBuffer genTestData(boolean useDirectBuffer, int sizeKB) {
+    Random random = new Random();
+    int bufferSize = sizeKB * 1024;
+    byte[] tmp = new byte[bufferSize];
+    random.nextBytes(tmp);
+    ByteBuffer data = useDirectBuffer ?
+        ByteBuffer.allocateDirect(bufferSize) :
+        ByteBuffer.allocate(bufferSize);
+    data.put(tmp);
+    data.flip();
+    return data;
+  }
+
+  private static class BenchData {
+    public static final ECReplicationConfig OPTIONS =
+        new ECReplicationConfig(6, 3);
+    public static final int NUM_DATA_UNITS = OPTIONS.getData();
+    public static final int NUM_PARITY_UNITS = OPTIONS.getParity();
+    public static final int NUM_ALL_UNITS = OPTIONS.getData() +
+        OPTIONS.getParity();
+    private static int chunkSize;
+    private static long totalDataSizeKB;
+    private static int bufferSizeKB;
+
+    private static final int[] ERASED_INDEXES = new int[]{6, 7, 8};
+    private final ByteBuffer[] inputs = new ByteBuffer[NUM_DATA_UNITS];
+    private ByteBuffer[] outputs = new ByteBuffer[NUM_PARITY_UNITS];
+    private ByteBuffer[] decodeInputs = new ByteBuffer[NUM_ALL_UNITS];
+
+    public static void configure(int dataSizeMB, int chunkSizeKB) {
+      chunkSize = chunkSizeKB * 1024;
+      // buffer size needs to be a multiple of (numDataUnits * chunkSize)
+      int round = (int) Math.round(
+          TARGET_BUFFER_SIZE_MB * 1024.0 / NUM_DATA_UNITS / chunkSizeKB);
+      Preconditions.checkArgument(round > 0);
+      bufferSizeKB = NUM_DATA_UNITS * chunkSizeKB * round;
+      System.out.println("Using " + bufferSizeKB / 1024 + "MB buffer.");
+
+      round = (int) Math.round(
+          (dataSizeMB * 1024.0) / bufferSizeKB);
+      if (round == 0) {
+        round = 1;
+      }
+      totalDataSizeKB = round * bufferSizeKB;
+    }
+
+    BenchData(boolean useDirectBuffer) {
+      for (int i = 0; i < outputs.length; i++) {
+        outputs[i] = useDirectBuffer ? ByteBuffer.allocateDirect(chunkSize) :
+            ByteBuffer.allocate(chunkSize);
+      }
+    }
+
+    public void prepareDecInput() {
+      System.arraycopy(inputs, 0, decodeInputs, 0, NUM_DATA_UNITS);
+    }
+
+    public void encode(RawErasureEncoder encoder) throws IOException {
+      encoder.encode(inputs, outputs);
+    }
+
+    public void decode(RawErasureDecoder decoder) throws IOException {
+      decoder.decode(decodeInputs, ERASED_INDEXES, outputs);
+    }
+  }
+
+  private static class BenchmarkCallable implements Callable<Long> {
+    private final boolean isEncode;
+    private final RawErasureEncoder encoder;
+    private final RawErasureDecoder decoder;
+    private final BenchData benchData;
+    private final ByteBuffer testData;
+
+    BenchmarkCallable(boolean isEncode, RawErasureEncoder encoder,
+        RawErasureDecoder decoder, ByteBuffer testData) {
+      if (isEncode) {
+        Preconditions.checkArgument(encoder != null);
+        this.encoder = encoder;
+        this.decoder = null;
+        benchData = new BenchData(encoder.preferDirectBuffer());
+      } else {
+        Preconditions.checkArgument(decoder != null);
+        this.decoder = decoder;
+        this.encoder = null;
+        benchData = new BenchData(decoder.preferDirectBuffer());
+      }
+      this.isEncode = isEncode;
+      this.testData = testData;
+    }
+
+    @Override
+    public Long call() throws Exception {
+      long rounds = BenchData.totalDataSizeKB / BenchData.bufferSizeKB;
+
+      StopWatch sw = new StopWatch().start();
+      for (long i = 0; i < rounds; i++) {
+        while (testData.remaining() > 0) {
+          for (ByteBuffer output : benchData.outputs) {
+            output.clear();
+          }
+
+          for (int j = 0; j < benchData.inputs.length; j++) {
+            benchData.inputs[j] = testData.duplicate();
+            benchData.inputs[j].limit(
+                testData.position() + BenchData.chunkSize);
+            benchData.inputs[j] = benchData.inputs[j].slice();
+            testData.position(testData.position() + BenchData.chunkSize);
+          }
+
+          if (isEncode) {
+            benchData.encode(encoder);
+          } else {
+            benchData.prepareDecInput();
+            benchData.decode(decoder);
+          }
+        }
+        testData.clear();
+      }
+      return sw.now(TimeUnit.MILLISECONDS);
+    }
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestCodecRawCoderMapping.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestCodecRawCoderMapping.java
new file mode 100644
index 0000000..5d749ee
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestCodecRawCoderMapping.java
@@ -0,0 +1,68 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.rawcoder.util.CodecUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test the codec to raw coder mapping.
+ */
+public class TestCodecRawCoderMapping {
+
+  private static final String RS_CODEC_NAME = "rs";
+  private static final String XOR_CODEC_NAME = "xor";
+
+  @Test
+  public void testRSDefaultRawCoder() {
+    ECReplicationConfig coderOptions =
+        new ECReplicationConfig(RS_CODEC_NAME + "-6-3-1024K");
+    // should return default raw coder of rs codec
+    RawErasureEncoder encoder =
+        CodecUtil.createRawEncoderWithFallback(coderOptions);
+    RawErasureDecoder decoder =
+        CodecUtil.createRawDecoderWithFallback(coderOptions);
+    if (ErasureCodeNative.isNativeCodeLoaded()) {
+      Assert.assertTrue(encoder instanceof NativeRSRawEncoder);
+      Assert.assertTrue(decoder instanceof NativeRSRawDecoder);
+    } else {
+      Assert.assertTrue(encoder instanceof RSRawEncoder);
+      Assert.assertTrue(decoder instanceof RSRawDecoder);
+    }
+  }
+
+  @Test
+  public void testXORRawCoder() {
+    ECReplicationConfig coderOptions =
+        new ECReplicationConfig(XOR_CODEC_NAME + "-6-3-1024K");
+    // should return default raw coder of rs codec
+    RawErasureEncoder encoder =
+        CodecUtil.createRawEncoderWithFallback(coderOptions);
+    RawErasureDecoder decoder =
+        CodecUtil.createRawDecoderWithFallback(coderOptions);
+    if (ErasureCodeNative.isNativeCodeLoaded()) {
+      Assert.assertTrue(encoder instanceof NativeXORRawEncoder);
+      Assert.assertTrue(decoder instanceof NativeXORRawDecoder);
+    } else {
+      Assert.assertTrue(encoder instanceof XORRawEncoder);
+      Assert.assertTrue(decoder instanceof XORRawDecoder);
+    }
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestDummyRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestDummyRawCoder.java
new file mode 100644
index 0000000..10c72b2
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestDummyRawCoder.java
@@ -0,0 +1,96 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.ozone.erasurecode.ECChunk;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Test dummy raw coder.
+ */
+public class TestDummyRawCoder extends TestRawCoderBase {
+
+  public TestDummyRawCoder() {
+    super(DummyRawErasureCoderFactory.class, DummyRawErasureCoderFactory.class);
+  }
+
+  @Before
+  public void setup() {
+    setAllowDump(false);
+    setChunkSize(baseChunkSize);
+  }
+
+  @Test
+  public void testCoding6x3ErasingD0Dd2() {
+    prepare(null, 6, 3, new int[]{0, 2}, new int[0], false);
+    testCodingDoMixed();
+  }
+
+  @Test
+  public void testCoding6x3ErasingD0P0() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0}, false);
+    testCodingDoMixed();
+  }
+
+  @Override
+  protected void testCoding(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders(true);
+
+    prepareBufferAllocator(true);
+
+    // Generate data and encode
+    ECChunk[] dataChunks = prepareDataChunksForEncoding();
+    markChunks(dataChunks);
+    ECChunk[] parityChunks = prepareParityChunksForEncoding();
+    try {
+      encode(dataChunks, parityChunks);
+    } catch (IOException e) {
+      Assert.fail("Unexpected IOException: " + e.getMessage());
+    }
+    compareAndVerify(parityChunks, getEmptyChunks(parityChunks.length));
+
+    // Decode
+    restoreChunksFromMark(dataChunks);
+    backupAndEraseChunks(dataChunks, parityChunks);
+    ECChunk[] inputChunks = prepareInputChunksForDecoding(
+        dataChunks, parityChunks);
+    ensureOnlyLeastRequiredChunks(inputChunks);
+    ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
+    try {
+      decode(inputChunks, getErasedIndexesForDecoding(),
+          recoveredChunks);
+    } catch (IOException e) {
+      Assert.fail("Unexpected IOException: " + e.getMessage());
+    }
+    compareAndVerify(recoveredChunks, getEmptyChunks(recoveredChunks.length));
+  }
+
+  private ECChunk[] getEmptyChunks(int num) {
+    ECChunk[] chunks = new ECChunk[num];
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = new ECChunk(ByteBuffer.wrap(getZeroChunkBytes()));
+    }
+    return chunks;
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeRSRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeRSRawCoder.java
new file mode 100644
index 0000000..2ab038b
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeRSRawCoder.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.ozone.erasurecode.rawcoder;
+
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test native raw Reed-solomon encoding and decoding.
+ */
+public class TestNativeRSRawCoder extends TestRSRawCoderBase {
+
+  public TestNativeRSRawCoder() {
+    super(NativeRSRawErasureCoderFactory.class,
+        NativeRSRawErasureCoderFactory.class);
+  }
+
+  @Before
+  public void setup() {
+    Assume.assumeTrue(ErasureCodeNative.isNativeCodeLoaded());
+    setAllowDump(true);
+  }
+
+  @Test
+  public void testCoding6x3ErasingAllD() {
+    prepare(null, 6, 3, new int[]{0, 1, 2}, new int[0], true);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasingD0D2() {
+    prepare(null, 6, 3, new int[] {0, 2}, new int[]{});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasingD0() {
+    prepare(null, 6, 3, new int[]{0}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasingD2() {
+    prepare(null, 6, 3, new int[]{2}, new int[]{});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasingD0P0() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasingAllP() {
+    prepare(null, 6, 3, new int[0], new int[]{0, 1, 2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasingP0() {
+    prepare(null, 6, 3, new int[0], new int[]{0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasingP2() {
+    prepare(null, 6, 3, new int[0], new int[]{2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasureP0P2() {
+    prepare(null, 6, 3, new int[0], new int[]{0, 2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasingD0P0P1() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0, 1});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6x3ErasingD0D2P2() {
+    prepare(null, 6, 3, new int[]{0, 2}, new int[]{2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingNegative6x3ErasingD2D4() {
+    prepare(null, 6, 3, new int[]{2, 4}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingNegative6x3ErasingTooMany() {
+    prepare(null, 6, 3, new int[]{2, 4}, new int[]{0, 1});
+    testCodingWithErasingTooMany();
+  }
+
+  @Test
+  public void testCoding10x4ErasingD0P0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testAfterRelease63() throws Exception {
+    prepare(6, 3, null, null);
+    testAfterRelease();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeXORRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeXORRawCoder.java
new file mode 100644
index 0000000..9312c25
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeXORRawCoder.java
@@ -0,0 +1,45 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test NativeXOR encoding and decoding.
+ */
+public class TestNativeXORRawCoder extends TestXORRawCoderBase {
+
+  public TestNativeXORRawCoder() {
+    super(NativeXORRawErasureCoderFactory.class,
+        NativeXORRawErasureCoderFactory.class);
+  }
+
+  @Before
+  public void setup() {
+    Assume.assumeTrue(ErasureCodeNative.isNativeCodeLoaded());
+    setAllowDump(true);
+  }
+
+  @Test
+  public void testAfterRelease63() throws Exception {
+    prepare(6, 3, null, null);
+    testAfterRelease();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoder.java
new file mode 100644
index 0000000..93496b3
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoder.java
@@ -0,0 +1,35 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.junit.Before;
+
+/**
+ * Test the new raw Reed-solomon coder implemented in Java.
+ */
+public class TestRSRawCoder extends TestRSRawCoderBase {
+
+  public TestRSRawCoder() {
+    super(RSRawErasureCoderFactory.class, RSRawErasureCoderFactory.class);
+  }
+
+  @Before
+  public void setup() {
+    setAllowDump(false);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoderBase.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoderBase.java
new file mode 100644
index 0000000..e6c795a
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoderBase.java
@@ -0,0 +1,123 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.junit.Test;
+
+/**
+ * Test base for raw Reed-solomon coders.
+ */
+public abstract class TestRSRawCoderBase extends TestRawCoderBase {
+
+  public TestRSRawCoderBase(
+      Class<? extends RawErasureCoderFactory> encoderFactoryClass,
+      Class<? extends RawErasureCoderFactory> decoderFactoryClass) {
+    super(encoderFactoryClass, decoderFactoryClass);
+  }
+
+  @Test
+  public void testCoding6xaErasingAllD() {
+    prepare(null, 6, 3, new int[]{0, 1, 2}, new int[0], true);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasingD0D2() {
+    prepare(null, 6, 3, new int[]{0, 2}, new int[]{});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasingD0() {
+    prepare(null, 6, 3, new int[]{0}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasingD2() {
+    prepare(null, 6, 3, new int[]{2}, new int[]{});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasingD0P0() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasingAllP() {
+    prepare(null, 6, 3, new int[0], new int[]{0, 1, 2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasingP0() {
+    prepare(null, 6, 3, new int[0], new int[]{0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasingP2() {
+    prepare(null, 6, 3, new int[0], new int[]{2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasureP0P2() {
+    prepare(null, 6, 3, new int[0], new int[]{0, 2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasingD0P0P1() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0, 1});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding6xaErasingD0D2P2() {
+    prepare(null, 6, 3, new int[]{0, 2}, new int[]{2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingNegative6xaErasingD2D4() {
+    prepare(null, 6, 3, new int[]{2, 4}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingNegative6xaErasingTooMany() {
+    prepare(null, 6, 3, new int[]{2, 4}, new int[]{0, 1});
+    testCodingWithErasingTooMany();
+  }
+
+  @Test
+  public void testCoding10x4ErasingD0P0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingInputBufferPosition() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0});
+    testInputPosition(false);
+    testInputPosition(true);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawCoderBase.java
new file mode 100644
index 0000000..044b60f
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawCoderBase.java
@@ -0,0 +1,353 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.ECChunk;
+import org.apache.ozone.erasurecode.TestCoderBase;
+import org.apache.ozone.test.LambdaTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Raw coder test base with utilities.
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public abstract class TestRawCoderBase extends TestCoderBase {
+  private final Class<? extends RawErasureCoderFactory> encoderFactoryClass;
+  private final Class<? extends RawErasureCoderFactory> decoderFactoryClass;
+  private RawErasureEncoder encoder;
+  private RawErasureDecoder decoder;
+
+  public TestRawCoderBase(
+      Class<? extends RawErasureCoderFactory> encoderFactoryClass,
+      Class<? extends RawErasureCoderFactory> decoderFactoryClass) {
+    this.encoderFactoryClass = encoderFactoryClass;
+    this.decoderFactoryClass = decoderFactoryClass;
+  }
+
+  /**
+   * Doing twice to test if the coders can be repeatedly reused. This matters
+   * as the underlying coding buffers are shared, which may have bugs.
+   */
+  protected void testCodingDoMixAndTwice() {
+    testCodingDoMixed();
+    testCodingDoMixed();
+  }
+
+  /**
+   * Doing in mixed buffer usage model to test if the coders can be repeatedly
+   * reused with different buffer usage model. This matters as the underlying
+   * coding buffers are shared, which may have bugs.
+   */
+  protected void testCodingDoMixed() {
+    testCoding(true);
+    testCoding(false);
+  }
+
+  /**
+   * Generating source data, encoding, recovering and then verifying.
+   * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
+   * it supports two kinds of ByteBuffers, one is array backed, the other is
+   * direct ByteBuffer. Use usingDirectBuffer indicate which case to test.
+   *
+   * @param usingDirectBuffer
+   */
+  protected void testCoding(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders(true);
+
+    /**
+     * The following runs will use 3 different chunkSize for inputs and outputs,
+     * to verify the same encoder/decoder can process variable width of data.
+     */
+    performTestCoding(baseChunkSize, true, false, false);
+    performTestCoding(baseChunkSize - 17, false, false, false);
+    performTestCoding(baseChunkSize + 16, true, false, false);
+  }
+
+  /**
+   * Similar to above, but perform negative cases using bad input for encoding.
+   * @param usingDirectBuffer
+   */
+  protected void testCodingWithBadInput(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders(true);
+
+    try {
+      performTestCoding(baseChunkSize, false, true, false);
+      Assert.fail("Encoding test with bad input should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  /**
+   * Similar to above, but perform negative cases using bad output for decoding.
+   * @param usingDirectBuffer
+   */
+  protected void testCodingWithBadOutput(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders(true);
+
+    try {
+      performTestCoding(baseChunkSize, false, false, true);
+      Assert.fail("Decoding test with bad output should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  /**
+   * Test encode / decode after release(). It should raise IOException.
+   *
+   * @throws Exception
+   */
+  void testAfterRelease() throws Exception {
+    prepareCoders(true);
+    prepareBufferAllocator(true);
+
+    encoder.release();
+    final ECChunk[] data = prepareDataChunksForEncoding();
+    final ECChunk[] parity = prepareParityChunksForEncoding();
+    LambdaTestUtils.intercept(IOException.class, "closed",
+        () -> encoder.encode(data, parity));
+
+    decoder.release();
+    final ECChunk[] in = prepareInputChunksForDecoding(data, parity);
+    final ECChunk[] out = prepareOutputChunksForDecoding();
+    LambdaTestUtils.intercept(IOException.class, "closed",
+        () -> decoder.decode(in, getErasedIndexesForDecoding(), out));
+  }
+
+  @Test
+  public void testCodingWithErasingTooMany() {
+    try {
+      testCoding(true);
+      Assert.fail("Decoding test erasing too many should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+
+    try {
+      testCoding(false);
+      Assert.fail("Decoding test erasing too many should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  @Test
+  public void testIdempotentReleases() {
+    prepareCoders(true);
+
+    for (int i = 0; i < 3; i++) {
+      encoder.release();
+      decoder.release();
+    }
+  }
+
+  private void performTestCoding(int chunkSize, boolean usingSlicedBuffer,
+      boolean useBadInput, boolean useBadOutput) {
+    setChunkSize(chunkSize);
+    prepareBufferAllocator(usingSlicedBuffer);
+
+    dumpSetting();
+
+    // Generate data and encode
+    ECChunk[] dataChunks = prepareDataChunksForEncoding();
+    if (useBadInput) {
+      corruptSomeChunk(dataChunks);
+    }
+    dumpChunks("Testing data chunks", dataChunks);
+
+    ECChunk[] parityChunks = prepareParityChunksForEncoding();
+
+    // Backup all the source chunks for later recovering because some coders
+    // may affect the source data.
+    ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks);
+    markChunks(dataChunks);
+
+    try {
+      encoder.encode(dataChunks, parityChunks);
+    } catch (IOException e) {
+      Assert.fail("Should not get IOException: " + e.getMessage());
+    }
+    dumpChunks("Encoded parity chunks", parityChunks);
+
+    //TODOif (!allowChangeInputs) {
+    restoreChunksFromMark(dataChunks);
+    compareAndVerify(clonedDataChunks, dataChunks);
+    //}
+
+    // Backup and erase some chunks
+    ECChunk[] backupChunks =
+        backupAndEraseChunks(clonedDataChunks, parityChunks);
+
+    // Decode
+    ECChunk[] inputChunks = prepareInputChunksForDecoding(
+        clonedDataChunks, parityChunks);
+
+    // Remove unnecessary chunks,
+    //     allowing only least required chunks to be read.
+    ensureOnlyLeastRequiredChunks(inputChunks);
+
+    ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
+    if (useBadOutput) {
+      corruptSomeChunk(recoveredChunks);
+    }
+
+    ECChunk[] clonedInputChunks = null;
+    //TODOif (!allowChangeInputs) {
+    markChunks(inputChunks);
+    clonedInputChunks = cloneChunksWithData(inputChunks);
+    //}
+
+    dumpChunks("Decoding input chunks", inputChunks);
+    try {
+      decoder.decode(inputChunks, getErasedIndexesForDecoding(),
+          recoveredChunks);
+    } catch (IOException e) {
+      Assert.fail("Should not get IOException: " + e.getMessage());
+    }
+    dumpChunks("Decoded/recovered chunks", recoveredChunks);
+
+    //TODOif (!allowChangeInputs) {
+    restoreChunksFromMark(inputChunks);
+    compareAndVerify(clonedInputChunks, inputChunks);
+    //}
+
+    // Compare
+    compareAndVerify(backupChunks, recoveredChunks);
+  }
+
+  /**
+   * Set true during setup if want to dump test settings and coding data,
+   * useful in debugging.
+   * @param allowDump
+   */
+  protected void setAllowDump(boolean allowDump) {
+    this.allowDump = allowDump;
+  }
+
+  protected void prepareCoders(boolean recreate) {
+    if (encoder == null || recreate) {
+      encoder = createEncoder();
+    }
+
+    if (decoder == null || recreate) {
+      decoder = createDecoder();
+    }
+  }
+
+  protected void ensureOnlyLeastRequiredChunks(ECChunk[] inputChunks) {
+    int leastRequiredNum = numDataUnits;
+    int erasedNum = erasedDataIndexes.length + erasedParityIndexes.length;
+    int goodNum = inputChunks.length - erasedNum;
+    int redundantNum = goodNum - leastRequiredNum;
+
+    for (int i = 0; i < inputChunks.length && redundantNum > 0; i++) {
+      if (inputChunks[i] != null) {
+        inputChunks[i] = null; // Setting it null, not needing it actually
+        redundantNum--;
+      }
+    }
+  }
+
+  /**
+   * Create the raw erasure encoder to test.
+   */
+  protected RawErasureEncoder createEncoder() {
+    ECReplicationConfig replicationConfig =
+        new ECReplicationConfig(numDataUnits, numParityUnits);
+    try {
+      RawErasureCoderFactory factory = encoderFactoryClass.newInstance();
+      return factory.createEncoder(replicationConfig);
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create encoder", e);
+    }
+  }
+
+  /**
+   * Create the raw erasure decoder to test.
+   */
+  protected RawErasureDecoder createDecoder() {
+    ECReplicationConfig replicationConfig =
+        new ECReplicationConfig(numDataUnits, numParityUnits);
+    try {
+      RawErasureCoderFactory factory = encoderFactoryClass.newInstance();
+      return factory.createDecoder(replicationConfig);
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create decoder", e);
+    }
+  }
+
+  /**
+   * Tests that the input buffer's position is moved to the end after
+   * encode/decode.
+   */
+  protected void testInputPosition(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders(true);
+    prepareBufferAllocator(false);
+
+    // verify encode
+    ECChunk[] dataChunks = prepareDataChunksForEncoding();
+    ECChunk[] parityChunks = prepareParityChunksForEncoding();
+    ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks);
+    try {
+      encoder.encode(dataChunks, parityChunks);
+    } catch (IOException e) {
+      Assert.fail("Should not get IOException: " + e.getMessage());
+    }
+    verifyBufferPositionAtEnd(dataChunks);
+
+    // verify decode
+    backupAndEraseChunks(clonedDataChunks, parityChunks);
+    ECChunk[] inputChunks = prepareInputChunksForDecoding(
+        clonedDataChunks, parityChunks);
+    ensureOnlyLeastRequiredChunks(inputChunks);
+    ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
+    try {
+      decoder.decode(inputChunks, getErasedIndexesForDecoding(),
+          recoveredChunks);
+    } catch (IOException e) {
+      Assert.fail("Should not get IOException: " + e.getMessage());
+    }
+    verifyBufferPositionAtEnd(inputChunks);
+  }
+
+  private void verifyBufferPositionAtEnd(ECChunk[] inputChunks) {
+    for (ECChunk chunk : inputChunks) {
+      if (chunk != null) {
+        Assert.assertEquals(0, chunk.getBuffer().remaining());
+      }
+    }
+  }
+
+  public void encode(ECChunk[] inputs, ECChunk[] outputs) throws IOException {
+    this.encoder.encode(inputs, outputs);
+  }
+
+  public void decode(ECChunk[] inputs, int[] erasedIndexes,
+      ECChunk[] outputs) throws IOException {
+    this.decoder.decode(inputs, erasedIndexes, outputs);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawErasureCoderBenchmark.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawErasureCoderBenchmark.java
new file mode 100644
index 0000000..cb700c4
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawErasureCoderBenchmark.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.junit.Test;
+
+/**
+ * Tests for the raw erasure coder benchmark tool.
+ */
+public class TestRawErasureCoderBenchmark {
+
+  @Test
+  public void testDummyCoder() throws Exception {
+    // Dummy coder
+    RawErasureCoderBenchmark.performBench("encode",
+        RawErasureCoderBenchmark.CODER.DUMMY_CODER, 2, 100, 1024);
+    RawErasureCoderBenchmark.performBench("decode",
+        RawErasureCoderBenchmark.CODER.DUMMY_CODER, 5, 150, 100);
+  }
+
+  @Test
+  public void testRSCoder() throws Exception {
+    // RS Java coder
+    RawErasureCoderBenchmark.performBench("encode",
+        RawErasureCoderBenchmark.CODER.RS_CODER, 3, 200, 200);
+    RawErasureCoderBenchmark.performBench("decode",
+        RawErasureCoderBenchmark.CODER.RS_CODER, 4, 135, 20);
+  }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoder.java
new file mode 100644
index 0000000..dcc8099
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoder.java
@@ -0,0 +1,30 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+/**
+ * Test pure Java XOR encoding and decoding.
+ */
+public class TestXORRawCoder extends TestXORRawCoderBase {
+
+  public TestXORRawCoder() {
+    super(XORRawErasureCoderFactory.class, XORRawErasureCoderFactory.class);
+  }
+
+
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoderBase.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoderBase.java
new file mode 100644
index 0000000..77ed8c7
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoderBase.java
@@ -0,0 +1,65 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.junit.Test;
+
+/**
+ * Test base for raw XOR coders.
+ */
+public abstract class TestXORRawCoderBase extends TestRawCoderBase {
+
+  public TestXORRawCoderBase(
+      Class<? extends RawErasureCoderFactory> encoderFactoryClass,
+      Class<? extends RawErasureCoderFactory> decoderFactoryClass) {
+    super(encoderFactoryClass, decoderFactoryClass);
+  }
+
+  @Test
+  public void testCoding10x1ErasingD0() {
+    prepare(null, 10, 1, new int[]{0}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding10x1ErasingP0() {
+    prepare(null, 10, 1, new int[0], new int[]{0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding10x1ErasingD5() {
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingNegative10x1ErasingTooMany() {
+    prepare(null, 10, 1, new int[]{2}, new int[]{0});
+    testCodingWithErasingTooMany();
+  }
+
+  @Test
+  public void testCodingNegative10x1ErasingD5() {
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+  }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/package-info.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/package-info.java
new file mode 100644
index 0000000..b2422af
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Unit tests for rawencoders.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
index f3c1da8..eb19e46 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
@@ -24,6 +24,7 @@
 
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
 import org.apache.hadoop.hdds.client.ContainerBlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
@@ -170,6 +171,13 @@
       requestBuilder.setFactor(
           ((RatisReplicationConfig) replicationConfig).getReplicationFactor());
       break;
+    case EC:
+      // We do not check for server support here, as this call is used only
+      // from OM which has the same software version as SCM.
+      // TODO: Rolling upgrade support needs to change this.
+      requestBuilder.setEcReplicationConfig(
+          ((ECReplicationConfig)replicationConfig).toProto());
+      break;
     default:
       throw new IllegalArgumentException(
           "Unsupported replication type " + replicationConfig
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
index beb3b75..568985e 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
@@ -21,6 +21,9 @@
 import com.google.protobuf.ServiceException;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicatedReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.GetScmInfoResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.UpgradeFinalizationStatus;
@@ -113,6 +116,8 @@
 import java.util.Optional;
 import java.util.function.Consumer;
 
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+
 /**
  * This class is the client-side translator to translate the requests made on
  * the {@link StorageContainerLocationProtocol} interface to the RPC server
@@ -357,18 +362,20 @@
   @Override
   public List<ContainerInfo> listContainer(long startContainerID, int count)
       throws IOException {
-    return listContainer(startContainerID, count, null, null);
+    return listContainer(startContainerID, count, null, null, null);
   }
 
   @Override
   public List<ContainerInfo> listContainer(long startContainerID, int count,
       HddsProtos.LifeCycleState state) throws IOException {
-    return listContainer(startContainerID, count, state, null);
+    return listContainer(startContainerID, count, state, null, null);
   }
 
   @Override
   public List<ContainerInfo> listContainer(long startContainerID, int count,
-      HddsProtos.LifeCycleState state, HddsProtos.ReplicationFactor factor)
+      HddsProtos.LifeCycleState state,
+      HddsProtos.ReplicationType replicationType,
+      ReplicationConfig replicationConfig)
       throws IOException {
     Preconditions.checkState(startContainerID >= 0,
         "Container ID cannot be negative.");
@@ -382,8 +389,18 @@
     if (state != null) {
       builder.setState(state);
     }
-    if (factor != null) {
-      builder.setFactor(factor);
+    if (replicationConfig != null) {
+      if (replicationConfig.getReplicationType() == EC) {
+        builder.setType(EC);
+        builder.setEcReplicationConfig(
+            ((ECReplicationConfig)replicationConfig).toProto());
+      } else {
+        builder.setType(replicationConfig.getReplicationType());
+        builder.setFactor(((ReplicatedReplicationConfig)replicationConfig)
+            .getReplicationFactor());
+      }
+    } else if (replicationType != null) {
+      builder.setType(replicationType);
     }
 
     SCMListContainerRequestProto request = builder.build();
@@ -400,6 +417,15 @@
     return containerList;
   }
 
+  @Deprecated
+  @Override
+  public List<ContainerInfo> listContainer(long startContainerID, int count,
+      HddsProtos.LifeCycleState state, HddsProtos.ReplicationFactor factor)
+      throws IOException {
+    throw new UnsupportedOperationException("Should no longer be called from " +
+        "the client side");
+  }
+
   /**
    * Ask SCM to delete a container by name. SCM will remove
    * the container mapping in its database.
diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
index 97e3dae..882f6c6 100644
--- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
+++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
@@ -265,6 +265,8 @@
   optional string traceID = 3;
   optional LifeCycleState state = 4;
   optional ReplicationFactor factor = 5;
+  optional ReplicationType type = 6;
+  optional ECReplicationConfig ecReplicationConfig = 7;
 }
 
 message SCMListContainerResponseProto {
diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto
index 01c9a1e..540ac7b 100644
--- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto
+++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto
@@ -155,6 +155,8 @@
   required int64 containerID = 1;
   required int64 localID = 2;
   optional uint64 blockCommitSequenceId = 3 [default = 0];
+  optional int32 replicaIndex = 4;
+
 }
 
 message KeyValue {
@@ -266,6 +268,7 @@
 message  CreateContainerRequestProto {
   repeated KeyValue metadata = 2;
   optional ContainerType containerType = 3 [default = KeyValueContainer];
+  optional int32 replicaIndex = 4;
 }
 
 message  CreateContainerResponseProto {
diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto
index caac440..c99274d 100644
--- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto
+++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto
@@ -125,6 +125,8 @@
     repeated uint32 memberOrders = 7;
     optional uint64 creationTimeStamp = 8;
     optional UUID suggestedLeaderID = 9;
+    repeated uint32 memberReplicaIndexes = 10;
+    optional ECReplicationConfig ecReplicationConfig = 11;
     // TODO(runzhiwang): when leaderID is gone, specify 6 as the index of leaderID128
     optional UUID leaderID128 = 100;
 }
@@ -230,8 +232,9 @@
     required string owner = 7;
     optional int64 deleteTransactionId = 8;
     optional int64 sequenceId = 9;
-    required ReplicationFactor replicationFactor  = 10;
+    optional ReplicationFactor replicationFactor  = 10;
     required ReplicationType replicationType  = 11;
+    optional ECReplicationConfig ecReplicationConfig = 12;
 }
 
 message ContainerWithPipeline {
@@ -264,11 +267,27 @@
     RATIS = 1;
     STAND_ALONE = 2;
     CHAINED = 3;
+    EC = 4;
+    NONE = -1; // Invalid Type
 }
 
 enum ReplicationFactor {
     ONE = 1;
     THREE = 3;
+    ZERO = 0; // Invalid Factor
+}
+
+message ECReplicationConfig {
+    required int32 data = 1;
+    required int32 parity = 2;
+    required string codec = 3;
+    required int32 ecChunkSize = 4;
+}
+
+message DefaultReplicationConfig {
+    required ReplicationType type = 1;
+    optional ReplicationFactor factor = 2;
+    optional ECReplicationConfig ecReplicationConfig = 3;
 }
 
 enum ScmOps {
@@ -394,6 +413,7 @@
     required int64 sequenceID = 5;
     required int64 keyCount = 6;
     required int64 bytesUsed = 7;
+    optional int64 replicaIndex = 8;
 }
 
 message KeyContainerIDList {
diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
index ffed754..7c07154 100644
--- a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
+++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
@@ -228,6 +228,7 @@
   optional int64 deleteTransactionId = 11;
   optional uint64 blockCommitSequenceId = 12;
   optional string originNodeId = 13;
+  optional int32 replicaIndex = 14;
 }
 
 message CommandStatusReportsProto {
diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto
index facdf58..90ce7f7 100644
--- a/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto
+++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto
@@ -139,9 +139,13 @@
   required uint64 size = 1;
   required uint32 numBlocks = 2;
   required ReplicationType type = 3;
-  required hadoop.hdds.ReplicationFactor factor = 4;
+  optional hadoop.hdds.ReplicationFactor factor = 4;
   required string owner = 5;
   optional ExcludeListProto excludeList = 7;
+
+  //used for EC replicaiton instead of the replication factor
+  optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 8;
+
 }
 
 /**
diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml
index f7480a2..35248b7 100644
--- a/hadoop-hdds/pom.xml
+++ b/hadoop-hdds/pom.xml
@@ -45,6 +45,7 @@
     <module>docs</module>
     <module>config</module>
     <module>test-utils</module>
+    <module>erasurecode</module>
   </modules>
 
   <repositories>
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
index 9b94e20..dd9f71a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdds.scm.container;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
@@ -132,7 +133,6 @@
                                     final ContainerInfo containerInfo,
                                     final ContainerReplicaProto replicaProto)
       throws ContainerNotFoundException {
-    final ContainerID containerId = containerInfo.containerID();
 
     if (isHealthy(replicaProto::getState)) {
       if (containerInfo.getSequenceId() <
@@ -140,30 +140,77 @@
         containerInfo.updateSequenceId(
             replicaProto.getBlockCommitSequenceId());
       }
-      List<ContainerReplica> otherReplicas =
-          getOtherReplicas(containerId, datanodeDetails);
-      long usedBytes = replicaProto.getUsed();
-      long keyCount = replicaProto.getKeyCount();
-      for (ContainerReplica r : otherReplicas) {
-        // Open containers are generally growing in key count and size, the
-        // overall size should be the min of all reported replicas.
-        if (containerInfo.getState().equals(HddsProtos.LifeCycleState.OPEN)) {
-          usedBytes = Math.min(usedBytes, r.getBytesUsed());
-          keyCount = Math.min(keyCount, r.getKeyCount());
-        } else {
-          // Containers which are not open can only shrink in size, so use the
-          // largest values reported.
-          usedBytes = Math.max(usedBytes, r.getBytesUsed());
-          keyCount = Math.max(keyCount, r.getKeyCount());
-        }
+      if (containerInfo.getReplicationConfig().getReplicationType()
+          == HddsProtos.ReplicationType.EC) {
+        updateECContainerStats(containerInfo, replicaProto, datanodeDetails);
+      } else {
+        updateRatisContainerStats(containerInfo, replicaProto, datanodeDetails);
       }
+    }
+  }
 
-      if (containerInfo.getUsedBytes() != usedBytes) {
-        containerInfo.setUsedBytes(usedBytes);
+  private void updateRatisContainerStats(ContainerInfo containerInfo,
+      ContainerReplicaProto newReplica, DatanodeDetails newSource)
+      throws ContainerNotFoundException {
+    List<ContainerReplica> otherReplicas =
+        getOtherReplicas(containerInfo.containerID(), newSource);
+    long usedBytes = newReplica.getUsed();
+    long keyCount = newReplica.getKeyCount();
+    for (ContainerReplica r : otherReplicas) {
+      usedBytes = calculateUsage(containerInfo, usedBytes, r.getBytesUsed());
+      keyCount = calculateUsage(containerInfo, keyCount, r.getKeyCount());
+    }
+    updateContainerUsedAndKeys(containerInfo, usedBytes, keyCount);
+  }
+
+  private void updateECContainerStats(ContainerInfo containerInfo,
+      ContainerReplicaProto newReplica, DatanodeDetails newSource)
+      throws ContainerNotFoundException {
+    int dataNum =
+        ((ECReplicationConfig)containerInfo.getReplicationConfig()).getData();
+    // The first EC index and the parity indexes must all be the same size
+    // while the other data indexes may be smaller due to partial stripes.
+    // When calculating the stats, we only use the first data and parity and
+    // ignore the others. We only need to run the check if we are processing
+    // the first data or parity replicas.
+    if (newReplica.getReplicaIndex() == 1
+        || newReplica.getReplicaIndex() > dataNum) {
+      List<ContainerReplica> otherReplicas =
+          getOtherReplicas(containerInfo.containerID(), newSource);
+      long usedBytes = newReplica.getUsed();
+      long keyCount = newReplica.getKeyCount();
+      for (ContainerReplica r : otherReplicas) {
+        if (r.getReplicaIndex() > 1 && r.getReplicaIndex() <= dataNum) {
+          // Ignore all data replicas except the first for stats
+          continue;
+        }
+        usedBytes = calculateUsage(containerInfo, usedBytes, r.getBytesUsed());
+        keyCount = calculateUsage(containerInfo, keyCount, r.getKeyCount());
       }
-      if (containerInfo.getNumberOfKeys() != keyCount) {
-        containerInfo.setNumberOfKeys(keyCount);
-      }
+      updateContainerUsedAndKeys(containerInfo, usedBytes, keyCount);
+    }
+  }
+
+  private long calculateUsage(ContainerInfo containerInfo, long lastValue,
+      long thisValue) {
+    if (containerInfo.getState().equals(HddsProtos.LifeCycleState.OPEN)) {
+      // Open containers are generally growing in key count and size, the
+      // overall size should be the min of all reported replicas.
+      return Math.min(lastValue, thisValue);
+    } else {
+      // Containers which are not open can only shrink in size, so use the
+      // largest values reported.
+      return Math.max(lastValue, thisValue);
+    }
+  }
+
+  private void updateContainerUsedAndKeys(ContainerInfo containerInfo,
+      long usedBytes, long keyCount) {
+    if (containerInfo.getUsedBytes() != usedBytes) {
+      containerInfo.setUsedBytes(usedBytes);
+    }
+    if (containerInfo.getNumberOfKeys() != keyCount) {
+      containerInfo.setNumberOfKeys(keyCount);
     }
   }
 
@@ -308,6 +355,7 @@
         .setOriginNodeId(UUID.fromString(replicaProto.getOriginNodeId()))
         .setSequenceId(replicaProto.getBlockCommitSequenceId())
         .setKeyCount(replicaProto.getKeyCount())
+        .setReplicaIndex(replicaProto.getReplicaIndex())
         .setBytesUsed(replicaProto.getUsed())
         .build();
 
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
index 1c39efd..e4076d9 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
@@ -35,6 +35,7 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto;
@@ -261,7 +262,8 @@
         "Cannot allocate container, negative container id" +
             " generated. %s.", uniqueId);
     final ContainerID containerID = ContainerID.valueOf(uniqueId);
-    final ContainerInfoProto containerInfo = ContainerInfoProto.newBuilder()
+    final ContainerInfoProto.Builder containerInfoBuilder = ContainerInfoProto
+        .newBuilder()
         .setState(LifeCycleState.OPEN)
         .setPipelineID(pipeline.getId().getProtobuf())
         .setUsedBytes(0)
@@ -270,11 +272,17 @@
         .setOwner(owner)
         .setContainerID(containerID.getId())
         .setDeleteTransactionId(0)
-        .setReplicationFactor(
-            ReplicationConfig.getLegacyFactor(pipeline.getReplicationConfig()))
-        .setReplicationType(pipeline.getType())
-        .build();
-    containerStateManager.addContainer(containerInfo);
+        .setReplicationType(pipeline.getType());
+
+    if (pipeline.getReplicationConfig() instanceof ECReplicationConfig) {
+      containerInfoBuilder.setEcReplicationConfig(
+          ((ECReplicationConfig) pipeline.getReplicationConfig()).toProto());
+    } else {
+      containerInfoBuilder.setReplicationFactor(
+          ReplicationConfig.getLegacyFactor(pipeline.getReplicationConfig()));
+    }
+
+    containerStateManager.addContainer(containerInfoBuilder.build());
     scmContainerManagerMetrics.incNumSuccessfulCreateContainers();
     return containerStateManager.getContainer(containerID);
   }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplica.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplica.java
index 0cb1bdd..727fdf8 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplica.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplica.java
@@ -37,21 +37,28 @@
   private final ContainerReplicaProto.State state;
   private final DatanodeDetails datanodeDetails;
   private final UUID placeOfBirth;
+  private final int replicaIndex;
 
   private Long sequenceId;
   private final long keyCount;
   private final long bytesUsed;
 
 
-  private ContainerReplica(final ContainerID containerID,
-      final ContainerReplicaProto.State state, final DatanodeDetails datanode,
-      final UUID originNodeId, long keyNum, long dataSize) {
+  private ContainerReplica(
+      final ContainerID containerID,
+      final ContainerReplicaProto.State state,
+      final int replicaIndex,
+      final DatanodeDetails datanode,
+      final UUID originNodeId,
+      long keyNum,
+      long dataSize) {
     this.containerID = containerID;
     this.state = state;
     this.datanodeDetails = datanode;
     this.placeOfBirth = originNodeId;
     this.keyCount = keyNum;
     this.bytesUsed = dataSize;
+    this.replicaIndex = replicaIndex;
   }
 
   private void setSequenceId(Long seqId) {
@@ -181,6 +188,7 @@
     private Long sequenceId;
     private long bytesUsed;
     private long keyCount;
+    private int replicaIndex;
 
     /**
      * Set Container Id.
@@ -212,6 +220,12 @@
       return this;
     }
 
+    public ContainerReplicaBuilder setReplicaIndex(
+        int index) {
+      this.replicaIndex = index;
+      return this;
+    }
+
     /**
      * Set replica origin node id.
      *
@@ -257,11 +271,15 @@
       Preconditions.checkNotNull(datanode,
           "DatanodeDetails can't be null");
       ContainerReplica replica = new ContainerReplica(
-          containerID, state, datanode,
+          containerID, state, replicaIndex, datanode,
           Optional.ofNullable(placeOfBirth).orElse(datanode.getUuid()),
           keyCount, bytesUsed);
       Optional.ofNullable(sequenceId).ifPresent(replica::setSequenceId);
       return replica;
     }
   }
+
+  public int getReplicaIndex() {
+    return replicaIndex;
+  }
 }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
index 6e49db9..4207482 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
@@ -477,6 +477,13 @@
           }
         }
 
+        if (container.getReplicationType() == HddsProtos.ReplicationType.EC) {
+          // TODO We do not support replicating EC containers as yet, so at this
+          //      point, after handing the closing etc states, we just return.
+          //      EC Support will be added later.
+          return;
+        }
+
         /*
          * Before processing the container we have to reconcile the
          * inflightReplication and inflightDeletion actions.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/ContainerPlacementPolicyFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/ContainerPlacementPolicyFactory.java
index bf0ea7c..9c64566 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/ContainerPlacementPolicyFactory.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/ContainerPlacementPolicyFactory.java
@@ -30,7 +30,8 @@
 
 /**
  * A factory to create container placement instance based on configuration
- * property {@link ScmConfigKeys#OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY}.
+ * property {@link ScmConfigKeys#OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY} and
+ * property {@link ScmConfigKeys#OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY}.
  */
 public final class ContainerPlacementPolicyFactory {
   private static final Logger LOG =
@@ -39,6 +40,9 @@
   private static final Class<? extends PlacementPolicy>
       OZONE_SCM_CONTAINER_PLACEMENT_IMPL_DEFAULT =
       SCMContainerPlacementRandom.class;
+  private static final Class<? extends PlacementPolicy>
+      OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_DEFAULT =
+      SCMContainerPlacementRackScatter.class;
 
   private ContainerPlacementPolicyFactory() {
   }
@@ -52,6 +56,28 @@
         .getClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
             OZONE_SCM_CONTAINER_PLACEMENT_IMPL_DEFAULT,
             PlacementPolicy.class);
+    return getPolicyInternal(placementClass, conf, nodeManager, clusterMap,
+        fallback, metrics);
+  }
+
+  public static PlacementPolicy getECPolicy(
+      ConfigurationSource conf, final NodeManager nodeManager,
+      NetworkTopology clusterMap, final boolean fallback,
+      SCMContainerPlacementMetrics metrics) throws SCMException {
+    // TODO: Change default placement policy for EC
+    final Class<? extends PlacementPolicy> placementClass = conf
+        .getClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY,
+            OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_DEFAULT,
+            PlacementPolicy.class);
+    return getPolicyInternal(placementClass, conf, nodeManager, clusterMap,
+        fallback, metrics);
+  }
+
+  private static PlacementPolicy getPolicyInternal(
+      Class<? extends PlacementPolicy> placementClass,
+      ConfigurationSource conf, final NodeManager nodeManager,
+      NetworkTopology clusterMap, final boolean fallback,
+      SCMContainerPlacementMetrics metrics) throws SCMException {
     Constructor<? extends PlacementPolicy> constructor;
     try {
       constructor = placementClass.getDeclaredConstructor(NodeManager.class,
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementMetrics.java
index 22bdf21..aaa7855 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementMetrics.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementMetrics.java
@@ -53,8 +53,18 @@
   public SCMContainerPlacementMetrics() {
   }
 
+  /**
+   * Return the existing instance of SCMContainerPlacementMetrics if it is
+   * registered in the metrics System, otherwise create a new instance, register
+   * it and return it.
+   * @return A new or existing SCMContainerPlacementMetrics object
+   */
   public static SCMContainerPlacementMetrics create() {
     MetricsSystem ms = DefaultMetricsSystem.instance();
+    MetricsSource existingSource = ms.getSource(SOURCE_NAME);
+    if (existingSource != null) {
+      return (SCMContainerPlacementMetrics)existingSource;
+    }
     registry = new MetricsRegistry(RECORD_INFO);
     return ms.register(SOURCE_NAME, "SCM Container Placement Metrics",
         new SCMContainerPlacementMetrics());
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java
new file mode 100644
index 0000000..1902d51
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java
@@ -0,0 +1,329 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.placement.algorithms;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.net.Node;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Container placement policy that scatter datanodes on different racks
+ * , together with the space to satisfy the size constraints.
+ * <p>
+ * This placement policy will try to distribute datanodes on as many racks as
+ * possible.
+ * <p>
+ * This implementation applies to network topology like "/rack/node". Don't
+ * recommend to use this if the network topology has more layers.
+ * <p>
+ */
+public final class SCMContainerPlacementRackScatter
+    extends SCMCommonPlacementPolicy {
+  @VisibleForTesting
+  public static final Logger LOG =
+      LoggerFactory.getLogger(SCMContainerPlacementRackScatter.class);
+  private final NetworkTopology networkTopology;
+  private static final int RACK_LEVEL = 1;
+  // OUTER_LOOP is to avoid infinity choose on all racks
+  private static final int OUTER_LOOP_MAX_RETRY = 3;
+  // INNER_LOOP is to choose node in each rack
+  private static final int INNER_LOOP_MAX_RETRY = 5;
+  private final SCMContainerPlacementMetrics metrics;
+
+  /**
+   * Constructs a Container Placement with rack awareness.
+   *
+   * @param nodeManager Node Manager
+   * @param conf Configuration
+   */
+  public SCMContainerPlacementRackScatter(final NodeManager nodeManager,
+      final ConfigurationSource conf, final NetworkTopology networkTopology,
+      boolean fallback, final SCMContainerPlacementMetrics metrics) {
+    super(nodeManager, conf);
+    this.networkTopology = networkTopology;
+    this.metrics = metrics;
+  }
+
+  /**
+   * Called by SCM to choose datanodes.
+   *
+   *
+   * @param excludedNodes - list of the datanodes to exclude.
+   * @param favoredNodes - list of nodes preferred. This is a hint to the
+   *                     allocator, whether the favored nodes will be used
+   *                     depends on whether the nodes meets the allocator's
+   *                     requirement.
+   * @param nodesRequired - number of datanodes required.
+   * @param dataSizeRequired - size required for the container.
+   * @param metadataSizeRequired - size required for Ratis metadata.
+   * @return List of datanodes.
+   * @throws SCMException  SCMException
+   */
+  @Override
+  public List<DatanodeDetails> chooseDatanodes(
+      List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes,
+      int nodesRequired, long metadataSizeRequired, long dataSizeRequired)
+      throws SCMException {
+    Preconditions.checkArgument(nodesRequired > 0);
+    metrics.incrDatanodeRequestCount(nodesRequired);
+    int excludedNodesCount = excludedNodes == null ? 0 : excludedNodes.size();
+    List<Node> availableNodes = networkTopology.getNodes(
+        networkTopology.getMaxLevel());
+    int totalNodesCount = availableNodes.size();
+    if (excludedNodes != null) {
+      availableNodes.removeAll(excludedNodes);
+    }
+    if (availableNodes.size() < nodesRequired) {
+      throw new SCMException("No enough datanodes to choose. " +
+          "TotalNode = " + totalNodesCount +
+          " AvailableNode = " + availableNodes.size() +
+          " RequiredNode = " + nodesRequired +
+          " ExcludedNode = " + excludedNodesCount, null);
+    }
+
+    List<DatanodeDetails> mutableFavoredNodes = new ArrayList<>();
+    if (favoredNodes != null) {
+      // Generate mutableFavoredNodes, only stores valid favoredNodes
+      for (DatanodeDetails datanodeDetails : favoredNodes) {
+        if (isValidNode(datanodeDetails, metadataSizeRequired,
+            dataSizeRequired)) {
+          mutableFavoredNodes.add(datanodeDetails);
+        }
+      }
+      Collections.shuffle(mutableFavoredNodes);
+    }
+    if (excludedNodes != null) {
+      mutableFavoredNodes.removeAll(excludedNodes);
+    }
+
+    // For excluded nodes, we sort their racks at rear
+    List<Node> racks = getAllRacks();
+    if (excludedNodes != null && excludedNodes.size() > 0) {
+      racks = sortRackWithExcludedNodes(racks, excludedNodes);
+    }
+
+    List<Node> toChooseRacks = new LinkedList<>(racks);
+    List<DatanodeDetails> chosenNodes = new ArrayList<>();
+    List<Node> unavailableNodes = new ArrayList<>();
+    Set<Node> skippedRacks = new HashSet<>();
+    if (excludedNodes != null) {
+      unavailableNodes.addAll(excludedNodes);
+    }
+
+    // If the result doesn't change after retryCount, we return with exception
+    int retryCount = 0;
+    while (nodesRequired > 0) {
+      if (retryCount > OUTER_LOOP_MAX_RETRY) {
+        throw new SCMException("No satisfied datanode to meet the" +
+            " excludedNodes and affinityNode constrains.", null);
+      }
+      int chosenListSize = chosenNodes.size();
+
+      // Refill toChooseRacks, we put skippedRacks in front of toChooseRacks
+      // for a even distribution
+      toChooseRacks.addAll(racks);
+      if (!skippedRacks.isEmpty()) {
+        toChooseRacks.removeAll(skippedRacks);
+        toChooseRacks.addAll(0, skippedRacks);
+        skippedRacks.clear();
+      }
+
+      if (mutableFavoredNodes.size() > 0) {
+        List<Node> chosenFavoredNodesInForLoop = new ArrayList<>();
+        for (DatanodeDetails favoredNode : mutableFavoredNodes) {
+          Node curRack = getRackOfDatanodeDetails(favoredNode);
+          if (toChooseRacks.contains(curRack)) {
+            chosenNodes.add(favoredNode);
+            toChooseRacks.remove(curRack);
+            chosenFavoredNodesInForLoop.add(favoredNode);
+            unavailableNodes.add(favoredNode);
+            nodesRequired--;
+            if (nodesRequired == 0) {
+              break;
+            }
+          }
+        }
+        mutableFavoredNodes.removeAll(chosenFavoredNodesInForLoop);
+      }
+
+      // If satisfied by favored nodes, return then.
+      if (nodesRequired == 0) {
+        break;
+      }
+
+      for (Node rack : toChooseRacks) {
+        if (rack == null) {
+          // TODO: need to recheck why null coming here.
+          continue;
+        }
+        Node node = chooseNode(rack.getNetworkFullPath(), unavailableNodes,
+            metadataSizeRequired, dataSizeRequired);
+        if (node != null) {
+          chosenNodes.add((DatanodeDetails) node);
+          mutableFavoredNodes.remove(node);
+          unavailableNodes.add(node);
+          nodesRequired--;
+          if (nodesRequired == 0) {
+            break;
+          }
+        } else {
+          // Store the skipped racks to check them first in next outer loop
+          skippedRacks.add(rack);
+        }
+      }
+      // Clear toChooseRacks for this loop
+      toChooseRacks.clear();
+
+      // If chosenNodes not changed, increase the retryCount
+      if (chosenListSize == chosenNodes.size()) {
+        retryCount++;
+      } else {
+        // If chosenNodes changed, reset the retryCount
+        retryCount = 0;
+      }
+    }
+    ContainerPlacementStatus placementStatus =
+        validateContainerPlacement(chosenNodes, nodesRequired);
+    if (!placementStatus.isPolicySatisfied()) {
+      LOG.warn("ContainerPlacementPolicy not met, currentRacks is {}," +
+          " desired racks is {}.", placementStatus.actualPlacementCount(),
+          placementStatus.expectedPlacementCount());
+    }
+    return chosenNodes;
+  }
+
+  @Override
+  public DatanodeDetails chooseNode(List<DatanodeDetails> healthyNodes) {
+    return null;
+  }
+
+  /**
+   * Choose a datanode which meets the requirements. If there is no node which
+   * meets all the requirements, there is fallback chosen process depending on
+   * whether fallback is allowed when this class is instantiated.
+   *
+   *
+   * @param scope - the rack we are searching nodes under
+   * @param excludedNodes - list of the datanodes to excluded. Can be null.
+   * @param dataSizeRequired - size required for the container.
+   * @param metadataSizeRequired - size required for Ratis metadata.
+   * @return the chosen datanode.
+   */
+  private Node chooseNode(String scope, List<Node> excludedNodes,
+      long metadataSizeRequired, long dataSizeRequired) {
+    int maxRetry = INNER_LOOP_MAX_RETRY;
+    while (true) {
+      metrics.incrDatanodeChooseAttemptCount();
+      Node node = networkTopology.chooseRandom(scope, excludedNodes);
+      if (node != null) {
+        DatanodeDetails datanodeDetails = (DatanodeDetails) node;
+        if (isValidNode(datanodeDetails, metadataSizeRequired,
+            dataSizeRequired)) {
+          metrics.incrDatanodeChooseSuccessCount();
+          return node;
+        }
+        // exclude the unavailable node for the following retries.
+        excludedNodes.add(node);
+      } else {
+        LOG.debug("Failed to find the datanode for container. excludedNodes: " +
+            "{}, rack {}", excludedNodes, scope);
+      }
+      maxRetry--;
+      if (maxRetry == 0) {
+        // avoid the infinite loop
+        LOG.info("No satisfied datanode to meet the constraints. "
+            + "Metadatadata size required: {} Data size required: {}, scope "
+            + "{}, excluded nodes {}",
+            metadataSizeRequired, dataSizeRequired, scope, excludedNodes);
+        return null;
+      }
+    }
+  }
+
+  /**
+   * For EC placement policy, desired rack count would be equal to the num of
+   * Replicas.
+   * @param numReplicas - num of Replicas.
+   * @return required rack count.
+   */
+  @Override
+  protected int getRequiredRackCount(int numReplicas) {
+    if (networkTopology == null) {
+      return 1;
+    }
+    int maxLevel = networkTopology.getMaxLevel();
+    int numRacks = networkTopology.getNumOfNodes(maxLevel - 1);
+    // Return the num of Rack if numRack less than numReplicas
+    return Math.min(numRacks, numReplicas);
+  }
+
+  private Node getRackOfDatanodeDetails(DatanodeDetails datanodeDetails) {
+    String location = datanodeDetails.getNetworkLocation();
+    return networkTopology.getNode(location);
+  }
+
+  /**
+   * For the rack holding excludedNodes, we prefer not to choose from these
+   * racks, so we sort these racks at rear.
+   * @param racks
+   * @param excludedNodes
+   * @return
+   */
+  private List<Node> sortRackWithExcludedNodes(List<Node> racks,
+      List<DatanodeDetails> excludedNodes) {
+    Set<Node> lessPreferredRacks = excludedNodes.stream()
+        .map(node -> networkTopology.getAncestor(node, RACK_LEVEL))
+        // Dead Nodes have been removed from the topology and so have a
+        // null rack. We need to exclude those from the rack list.
+        .filter(node -> node != null)
+        .collect(Collectors.toSet());
+    List <Node> result = new ArrayList<>();
+    for (Node rack : racks) {
+      if (!lessPreferredRacks.contains(rack)) {
+        result.add(rack);
+      }
+    }
+    result.addAll(lessPreferredRacks);
+    return result;
+  }
+
+  private List<Node> getAllRacks() {
+    int rackLevel = networkTopology.getMaxLevel() - 1;
+    List<Node> racks = networkTopology.getNodes(rackLevel);
+    Collections.shuffle(racks);
+    return racks;
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java
index 75abd53..6fed1e2 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java
@@ -213,6 +213,9 @@
         new ArrayList<>();
     for (HddsProtos.ReplicationFactor factor : HddsProtos.ReplicationFactor
         .values()) {
+      if (factor == ReplicationFactor.ZERO) {
+        continue; // Ignore it.
+      }
       final ReplicationConfig replicationConfig =
           ReplicationConfig.fromProtoTypeAndFactor(type, factor);
       if (skipCreation(replicationConfig, autoCreateFactorOne)) {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/ECPipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/ECPipelineProvider.java
new file mode 100644
index 0000000..11c23f1
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/ECPipelineProvider.java
@@ -0,0 +1,126 @@
+/**
+ * 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.hdds.scm.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class to create pipelines for EC containers.
+ */
+public class ECPipelineProvider extends PipelineProvider<ECReplicationConfig> {
+
+  // TODO - EC Placement Policy. Standard Network Aware topology will not work
+  //        for EC as it stands. We may want an "as many racks as possible"
+  //        policy. HDDS-5326.
+
+  private final ConfigurationSource conf;
+  private final PlacementPolicy placementPolicy;
+  private final long containerSizeBytes;
+
+  public ECPipelineProvider(NodeManager nodeManager,
+                            PipelineStateManager stateManager,
+                            ConfigurationSource conf,
+                            PlacementPolicy placementPolicy) {
+    super(nodeManager, stateManager);
+    this.conf = conf;
+    this.placementPolicy = placementPolicy;
+    this.containerSizeBytes = (long) this.conf
+        .getStorageSize(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+            ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
+  }
+
+  @Override
+  public synchronized Pipeline create(ECReplicationConfig replicationConfig)
+      throws IOException {
+    return create(replicationConfig, Collections.emptyList(),
+        Collections.emptyList());
+  }
+
+  @Override
+  protected Pipeline create(ECReplicationConfig replicationConfig,
+      List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+      throws IOException {
+    List<DatanodeDetails> dns = placementPolicy
+        .chooseDatanodes(excludedNodes, favoredNodes,
+            replicationConfig.getRequiredNodes(), 0, this.containerSizeBytes);
+    return create(replicationConfig, dns);
+  }
+
+  @Override
+  protected Pipeline create(ECReplicationConfig replicationConfig,
+      List<DatanodeDetails> nodes) {
+
+    Map<DatanodeDetails, Integer> dnIndexes = new HashMap<>();
+    int ecIndex = 1;
+    for (DatanodeDetails dn : nodes) {
+      dnIndexes.put(dn, ecIndex);
+      ecIndex++;
+    }
+
+    return createPipelineInternal(replicationConfig, nodes, dnIndexes);
+  }
+
+  @Override
+  public Pipeline createForRead(
+      ECReplicationConfig replicationConfig,
+      Set<ContainerReplica> replicas) {
+    Map<DatanodeDetails, Integer> map = new HashMap<>();
+    List<DatanodeDetails> dns = new ArrayList<>(replicas.size());
+
+    for (ContainerReplica r : replicas) {
+      map.put(r.getDatanodeDetails(), r.getReplicaIndex());
+      dns.add(r.getDatanodeDetails());
+    }
+    return createPipelineInternal(replicationConfig, dns, map);
+  }
+
+  private Pipeline createPipelineInternal(ECReplicationConfig repConfig,
+      List<DatanodeDetails> dns, Map<DatanodeDetails, Integer> indexes) {
+    return Pipeline.newBuilder()
+        .setId(PipelineID.randomId())
+        .setState(Pipeline.PipelineState.ALLOCATED)
+        .setReplicationConfig(repConfig)
+        .setNodes(dns)
+        .setReplicaIndexes(indexes)
+        .build();
+  }
+
+  @Override
+  protected void close(Pipeline pipeline) throws IOException {
+  }
+
+  @Override
+  protected void shutdown() {
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java
index b58716b..780a4ee 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java
@@ -23,6 +23,11 @@
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicyFactory;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementMetrics;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.ha.SCMContext;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
@@ -32,6 +37,7 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * Creates pipeline based on replication type.
@@ -50,6 +56,18 @@
         new RatisPipelineProvider(nodeManager,
             stateManager, conf,
             eventPublisher, scmContext));
+    PlacementPolicy ecPlacementPolicy;
+    try {
+      ecPlacementPolicy = ContainerPlacementPolicyFactory.getECPolicy(conf,
+          nodeManager, nodeManager.getClusterNetworkTopologyMap(), true,
+          SCMContainerPlacementMetrics.create());
+    } catch (SCMException e) {
+      throw new RuntimeException("Unable to get the container placement policy",
+          e);
+    }
+    providers.put(ReplicationType.EC,
+        new ECPipelineProvider(nodeManager, stateManager, conf,
+            ecPlacementPolicy));
   }
 
   protected PipelineFactory() {
@@ -64,12 +82,12 @@
   }
 
   public Pipeline create(
-      ReplicationConfig replicationConfig
-  )
+      ReplicationConfig replicationConfig, List<DatanodeDetails> excludedNodes,
+      List<DatanodeDetails> favoredNodes)
       throws IOException {
     return providers
         .get(replicationConfig.getReplicationType())
-        .create(replicationConfig);
+        .create(replicationConfig, excludedNodes, favoredNodes);
   }
 
   public Pipeline create(ReplicationConfig replicationConfig,
@@ -79,6 +97,12 @@
         .create(replicationConfig, nodes);
   }
 
+  public Pipeline createForRead(ReplicationConfig replicationConfig,
+      Set<ContainerReplica> replicas) {
+    return providers.get(replicationConfig.getReplicationType())
+        .createForRead(replicationConfig, replicas);
+  }
+
   public void close(ReplicationType type, Pipeline pipeline)
       throws IOException {
     providers.get(type).close(pipeline);
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java
index b96d616..6a50876 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java
@@ -23,10 +23,12 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.NavigableSet;
+import java.util.Set;
 
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.utils.db.Table;
 
 /**
@@ -40,10 +42,18 @@
       throws IOException;
 
   Pipeline createPipeline(
+      ReplicationConfig replicationConfig, List<DatanodeDetails> excludedNodes,
+      List<DatanodeDetails> favoredNodes) throws IOException;
+
+
+  Pipeline createPipeline(
       ReplicationConfig replicationConfig,
       List<DatanodeDetails> nodes
   );
 
+  Pipeline createPipelineForRead(
+      ReplicationConfig replicationConfig, Set<ContainerReplica> replicas);
+
   Pipeline getPipeline(PipelineID pipelineID) throws PipelineNotFoundException;
 
   boolean containsPipeline(PipelineID pipelineID);
@@ -65,6 +75,17 @@
       Collection<PipelineID> excludePipelines
   );
 
+  /**
+   * Returns the count of pipelines meeting the given ReplicationConfig and
+   * state.
+   * @param replicationConfig The ReplicationConfig of the pipelines to count
+   * @param state The current state of the pipelines to count
+   * @return The count of pipelines meeting the above criteria
+   */
+  int getPipelineCount(
+      ReplicationConfig replicationConfig, Pipeline.PipelineState state
+  );
+
   void addContainerToPipeline(PipelineID pipelineID, ContainerID containerID)
       throws IOException;
 
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java
index 2168138..19b7a51 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java
@@ -31,6 +31,7 @@
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.ha.SCMContext;
 import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
@@ -51,6 +52,7 @@
 import java.time.Duration;
 import java.time.Instant;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -151,6 +153,14 @@
   public Pipeline createPipeline(
       ReplicationConfig replicationConfig
   ) throws IOException {
+    return createPipeline(replicationConfig, Collections.emptyList(),
+        Collections.emptyList());
+  }
+
+  @Override
+  public Pipeline createPipeline(ReplicationConfig replicationConfig,
+      List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+      throws IOException {
     if (!isPipelineCreationAllowed() && !factorOne(replicationConfig)) {
       LOG.debug("Pipeline creation is not allowed until safe mode prechecks " +
           "complete");
@@ -167,7 +177,8 @@
 
     acquireWriteLock();
     try {
-      Pipeline pipeline = pipelineFactory.create(replicationConfig);
+      Pipeline pipeline = pipelineFactory.create(replicationConfig,
+          excludedNodes, favoredNodes);
       stateManager.addPipeline(pipeline.getProtobufMessage(
           ClientVersion.CURRENT_VERSION));
       recordMetricsForPipeline(pipeline);
@@ -207,6 +218,12 @@
   }
 
   @Override
+  public Pipeline createPipelineForRead(
+      ReplicationConfig replicationConfig, Set<ContainerReplica> replicas) {
+    return pipelineFactory.createForRead(replicationConfig, replicas);
+  }
+
+  @Override
   public Pipeline getPipeline(PipelineID pipelineID)
       throws PipelineNotFoundException {
     return stateManager.getPipeline(pipelineID);
@@ -248,6 +265,19 @@
   }
 
   @Override
+  /**
+   * Returns the count of pipelines meeting the given ReplicationConfig and
+   * state.
+   * @param replicationConfig The ReplicationConfig of the pipelines to count
+   * @param state The current state of the pipelines to count
+   * @return The count of pipelines meeting the above criteria
+   */
+  public int getPipelineCount(ReplicationConfig config,
+                                     Pipeline.PipelineState state) {
+    return stateManager.getPipelineCount(config, state);
+  }
+
+  @Override
   public void addContainerToPipeline(
       PipelineID pipelineID, ContainerID containerID) throws IOException {
     // should not lock here, since no ratis operation happens.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
index 3c00906..063ca76 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
@@ -27,6 +27,7 @@
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.node.NodeStatus;
@@ -65,11 +66,20 @@
   protected abstract Pipeline create(REPLICATION_CONFIG replicationConfig)
       throws IOException;
 
+  protected abstract Pipeline create(REPLICATION_CONFIG replicationConfig,
+      List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+      throws IOException;
+
   protected abstract Pipeline create(
       REPLICATION_CONFIG replicationConfig,
       List<DatanodeDetails> nodes
   );
 
+  protected abstract Pipeline createForRead(
+      REPLICATION_CONFIG replicationConfig,
+      Set<ContainerReplica> replicas
+  );
+
   protected abstract void close(Pipeline pipeline) throws IOException;
 
   protected abstract void shutdown();
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
index 4af64b5..6180158 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
@@ -95,6 +95,11 @@
       Collection<PipelineID> excludePipelines
   );
 
+  int getPipelineCount(
+      ReplicationConfig replicationConfig,
+      Pipeline.PipelineState state
+  );
+
   NavigableSet<ContainerID> getContainers(PipelineID pipelineID)
       throws IOException;
 
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManagerImpl.java
index 0b93f4d..ff00e3b 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManagerImpl.java
@@ -188,6 +188,26 @@
     }
   }
 
+
+  /**
+   * Returns the count of pipelines meeting the given ReplicationConfig and
+   * state.
+   * @param replicationConfig The ReplicationConfig of the pipelines to count
+   * @param state The current state of the pipelines to count
+   * @return The count of pipelines meeting the above criteria
+   */
+  @Override
+  public int getPipelineCount(
+      ReplicationConfig replicationConfig,
+      Pipeline.PipelineState state) {
+    lock.readLock().lock();
+    try {
+      return pipelineStateMap.getPipelineCount(replicationConfig, state);
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
   @Override
   public NavigableSet<ContainerID> getContainers(PipelineID pipelineID)
       throws IOException {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java
index 8b9d913..23351e5 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java
@@ -220,6 +220,36 @@
   }
 
   /**
+   * Get a count of pipelines with the given replicationConfig and state.
+   * This method is most efficient when getting a count for OPEN pipeline
+   * as the result can be obtained directly from the cached open list.
+   *
+   * @param replicationConfig - ReplicationConfig
+   * @param state             - Required PipelineState
+   * @return Count of pipelines with the specified replication config and state
+   */
+  int getPipelineCount(ReplicationConfig replicationConfig,
+      PipelineState state) {
+    Preconditions
+        .checkNotNull(replicationConfig, "ReplicationConfig cannot be null");
+    Preconditions.checkNotNull(state, "Pipeline state cannot be null");
+
+    if (state == PipelineState.OPEN) {
+      return query2OpenPipelines.getOrDefault(
+              replicationConfig, Collections.EMPTY_LIST).size();
+    }
+
+    int count = 0;
+    for (Pipeline pipeline : pipelineMap.values()) {
+      if (pipeline.getReplicationConfig().equals(replicationConfig)
+          && pipeline.getPipelineState() == state) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  /**
    * Get list of pipeline corresponding to specified replication type,
    * replication factor and pipeline state.
    *
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
index 4c18f20..43b2e01 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
@@ -19,7 +19,10 @@
 package org.apache.hadoop.hdds.scm.pipeline;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -27,6 +30,7 @@
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.ha.SCMContext;
@@ -134,6 +138,14 @@
   @Override
   public synchronized Pipeline create(RatisReplicationConfig replicationConfig)
       throws IOException {
+    return create(replicationConfig, Collections.emptyList(),
+        Collections.emptyList());
+  }
+
+  @Override
+  public synchronized Pipeline create(RatisReplicationConfig replicationConfig,
+      List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+      throws IOException {
     if (exceedPipelineNumberLimit(replicationConfig)) {
       throw new SCMException("Ratis pipeline number meets the limit: " +
           pipelineNumberLimit + " replicationConfig : " +
@@ -151,8 +163,8 @@
           containerSizeBytes);
       break;
     case THREE:
-      dns = placementPolicy.chooseDatanodes(null,
-          null, factor.getNumber(), minRatisVolumeSizeBytes,
+      dns = placementPolicy.chooseDatanodes(excludedNodes,
+          favoredNodes, factor.getNumber(), minRatisVolumeSizeBytes,
           containerSizeBytes);
       break;
     default:
@@ -201,6 +213,16 @@
   }
 
   @Override
+  public Pipeline createForRead(
+      RatisReplicationConfig replicationConfig,
+      Set<ContainerReplica> replicas) {
+    return create(replicationConfig, replicas
+        .stream()
+        .map(ContainerReplica::getDatanodeDetails)
+        .collect(Collectors.toList()));
+  }
+
+  @Override
   public void shutdown() {
   }
 
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
index 43af98b..a61ed27 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
@@ -20,12 +20,15 @@
 
 import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState;
 
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 /**
  * Implements Api for creating stand alone pipelines.
@@ -41,6 +44,14 @@
   @Override
   public Pipeline create(StandaloneReplicationConfig replicationConfig)
       throws IOException {
+    return create(replicationConfig, Collections.emptyList(),
+        Collections.emptyList());
+  }
+
+  @Override
+  public Pipeline create(StandaloneReplicationConfig replicationConfig,
+      List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+      throws IOException {
     List<DatanodeDetails> dns = pickNodesNotUsed(replicationConfig);
     if (dns.size() < replicationConfig.getRequiredNodes()) {
       String e = String
@@ -71,6 +82,15 @@
   }
 
   @Override
+  public Pipeline createForRead(StandaloneReplicationConfig replicationConfig,
+      Set<ContainerReplica> replicas) {
+    return create(replicationConfig, replicas
+        .stream()
+        .map(ContainerReplica::getDatanodeDetails)
+        .collect(Collectors.toList()));
+  }
+
+  @Override
   public void close(Pipeline pipeline) throws IOException {
 
   }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java
index f130eed..7d25ee9 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdds.scm.pipeline;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
@@ -33,12 +34,16 @@
 
   private final WritableContainerProvider<ReplicationConfig> ratisProvider;
   private final WritableContainerProvider<ReplicationConfig> standaloneProvider;
+  private final WritableContainerProvider<ECReplicationConfig> ecProvider;
 
   public WritableContainerFactory(StorageContainerManager scm) {
     this.ratisProvider = new WritableRatisContainerProvider(
         scm.getConfiguration(), scm.getPipelineManager(),
         scm.getContainerManager(), scm.getPipelineChoosePolicy());
     this.standaloneProvider = ratisProvider;
+    this.ecProvider = new WritableECContainerProvider(scm.getConfiguration(),
+        scm.getPipelineManager(), scm.getContainerManager(),
+        scm.getPipelineChoosePolicy());
   }
 
   public ContainerInfo getContainer(final long size,
@@ -50,6 +55,9 @@
           .getContainer(size, repConfig, owner, excludeList);
     case RATIS:
       return ratisProvider.getContainer(size, repConfig, owner, excludeList);
+    case EC:
+      return ecProvider.getContainer(size, (ECReplicationConfig)repConfig,
+          owner, excludeList);
     default:
       throw new IOException(repConfig.getReplicationType()
           + " is an invalid replication type");
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java
new file mode 100644
index 0000000..117d957
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java
@@ -0,0 +1,239 @@
+/**
+ * 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.hdds.scm.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.Config;
+import org.apache.hadoop.hdds.conf.ConfigGroup;
+import org.apache.hadoop.hdds.conf.ConfigTag;
+import org.apache.hadoop.hdds.conf.ConfigType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.PipelineChoosePolicy;
+import org.apache.hadoop.hdds.scm.PipelineRequestInformation;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.NavigableSet;
+
+import static org.apache.hadoop.hdds.conf.StorageUnit.BYTES;
+
+/**
+ * Writable Container provider to obtain a writable container for EC pipelines.
+ */
+public class WritableECContainerProvider
+    implements WritableContainerProvider<ECReplicationConfig> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(WritableECContainerProvider.class);
+
+  private final ConfigurationSource conf;
+  private final PipelineManager pipelineManager;
+  private final PipelineChoosePolicy pipelineChoosePolicy;
+  private final ContainerManager containerManager;
+  private final long containerSize;
+  private final WritableECContainerProviderConfig providerConfig;
+
+  public WritableECContainerProvider(ConfigurationSource conf,
+      PipelineManager pipelineManager, ContainerManager containerManager,
+      PipelineChoosePolicy pipelineChoosePolicy) {
+    this.conf = conf;
+    this.providerConfig =
+        conf.getObject(WritableECContainerProviderConfig.class);
+    this.pipelineManager = pipelineManager;
+    this.containerManager = containerManager;
+    this.pipelineChoosePolicy = pipelineChoosePolicy;
+    this.containerSize = getConfiguredContainerSize();
+  }
+
+  /**
+   *
+   * @param size The max size of block in bytes which will be written. This
+   *             comes from Ozone Manager and will be the block size configured
+   *             for the cluster. The client cannot pass any arbitrary value
+   *             from this setting.
+   * @param repConfig The replication Config indicating the EC data and partiy
+   *                  block counts.
+   * @param owner The owner of the container
+   * @param excludeList A set of datanodes, container and pipelines which should
+   *                    not be considered.
+   * @return A containerInfo representing a block group with with space for the
+   *         write, or null if no container can be allocated.
+   * @throws IOException
+   */
+  @Override
+  public ContainerInfo getContainer(final long size,
+      ECReplicationConfig repConfig, String owner, ExcludeList excludeList)
+      throws IOException {
+    // Bound this at a minimum of 1 byte in case a request is made for a very
+    // small size, which when divided by EC DataNum is zero.
+    long requiredSpace = Math.max(1, size / repConfig.getData());
+    synchronized (this) {
+      int openPipelineCount = pipelineManager.getPipelineCount(repConfig,
+          Pipeline.PipelineState.OPEN);
+      if (openPipelineCount < providerConfig.getMinimumPipelines()) {
+        try {
+          return allocateContainer(
+              repConfig, requiredSpace, owner, excludeList);
+        } catch (IOException e) {
+          LOG.warn("Unable to allocate a container for {} with {} existing "
+              + "containers", repConfig, openPipelineCount, e);
+        }
+      }
+    }
+    List<Pipeline> existingPipelines = pipelineManager.getPipelines(
+        repConfig, Pipeline.PipelineState.OPEN,
+        excludeList.getDatanodes(), excludeList.getPipelineIds());
+
+    PipelineRequestInformation pri =
+        PipelineRequestInformation.Builder.getBuilder()
+            .setSize(requiredSpace)
+            .build();
+    while (existingPipelines.size() > 0) {
+      Pipeline pipeline =
+          pipelineChoosePolicy.choosePipeline(existingPipelines, pri);
+      if (pipeline == null) {
+        LOG.warn("Unable to select a pipeline from {} in the list",
+            existingPipelines.size());
+        break;
+      }
+      synchronized (pipeline.getId()) {
+        try {
+          ContainerInfo containerInfo = getContainerFromPipeline(pipeline);
+          if (containerInfo == null
+              || !containerHasSpace(containerInfo, requiredSpace)) {
+            // This is O(n), which isn't great if there are a lot of pipelines
+            // and we keep finding pipelines without enough space.
+            existingPipelines.remove(pipeline);
+            pipelineManager.closePipeline(pipeline, true);
+          } else {
+            if (containerIsExcluded(containerInfo, excludeList)) {
+              existingPipelines.remove(pipeline);
+            } else {
+              return containerInfo;
+            }
+          }
+        } catch (PipelineNotFoundException | ContainerNotFoundException e) {
+          LOG.warn("Pipeline or container not found when selecting a writable "
+              + "container", e);
+          existingPipelines.remove(pipeline);
+          pipelineManager.closePipeline(pipeline, true);
+        }
+      }
+    }
+    // If we get here, all the pipelines we tried were no good. So try to
+    // allocate a new one and usePipelineManagerV2Impl.java it.
+    try {
+      synchronized (this) {
+        return allocateContainer(repConfig, requiredSpace, owner, excludeList);
+      }
+    } catch (IOException e) {
+      LOG.error("Unable to allocate a container for {} after trying all "
+          + "existing containers", repConfig, e);
+      return null;
+    }
+  }
+
+  private ContainerInfo allocateContainer(ReplicationConfig repConfig,
+      long size, String owner, ExcludeList excludeList) throws IOException {
+
+    List<DatanodeDetails> excludedNodes = Collections.emptyList();
+    if (excludeList.getDatanodes().size() > 0) {
+      excludedNodes = new ArrayList<>(excludeList.getDatanodes());
+    }
+
+    Pipeline newPipeline = pipelineManager.createPipeline(repConfig,
+        excludedNodes, Collections.emptyList());
+    ContainerInfo container =
+        containerManager.getMatchingContainer(size, owner, newPipeline);
+    pipelineManager.openPipeline(newPipeline.getId());
+    return container;
+  }
+
+  private boolean containerIsExcluded(ContainerInfo container,
+      ExcludeList excludeList) {
+    return excludeList.getContainerIds().contains(container.containerID());
+  }
+
+  private ContainerInfo getContainerFromPipeline(Pipeline pipeline)
+      throws IOException {
+    // Assume the container is still open if the below method returns it. On
+    // container FINALIZE, ContainerManager will remove the container from the
+    // pipeline list in PipelineManager. Finalize can be triggered by a DN
+    // sending a message that the container is full, or on close pipeline or
+    // on a stale / dead node event (via close pipeline).
+    NavigableSet<ContainerID> containers =
+        pipelineManager.getContainersInPipeline(pipeline.getId());
+    // Assume 1 container per pipeline for EC
+    if (containers.size() == 0) {
+      return null;
+    }
+    ContainerID containerID = containers.first();
+    return containerManager.getContainer(containerID);
+  }
+
+  private boolean containerHasSpace(ContainerInfo container, long size) {
+    // The size passed from OM will be the cluster block size. Therefore we
+    // just check if the container has enough free space to accommodate another
+    // full block.
+    return container.getUsedBytes() + size <= containerSize;
+  }
+
+  private long getConfiguredContainerSize() {
+    return (long) conf.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, BYTES);
+  }
+
+  /**
+   * Class to hold configuration for WriteableECContainerProvider.
+   */
+  @ConfigGroup(prefix = "ozone.scm.ec")
+  public static class WritableECContainerProviderConfig {
+
+    @Config(key = "pipeline.minimum",
+        defaultValue = "5",
+        type = ConfigType.INT,
+        description = "The minimum number of pipelines to have open for each " +
+            "Erasure Coding configuration",
+        tags = ConfigTag.STORAGE)
+    private int minimumPipelines = 5;
+
+    public int getMinimumPipelines() {
+      return minimumPipelines;
+    }
+
+    public void setMinimumPipelines(int minPipelines) {
+      this.minimumPipelines = minPipelines;
+    }
+
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java
index 7b0fc67..bdf248b 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java
@@ -47,6 +47,7 @@
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher;
+import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
 import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
@@ -127,6 +128,17 @@
     try {
       switch (request.getCmdType()) {
       case AllocateScmBlock:
+        if (scm.getLayoutVersionManager().needsFinalization() &&
+            !scm.getLayoutVersionManager()
+                .isAllowed(HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)
+        ) {
+          if (request.getAllocateScmBlockRequest().hasEcReplicationConfig()) {
+            throw new SCMException("Cluster is not finalized yet, it is"
+                + " not enabled to create blocks with Erasure Coded"
+                + " replication type.",
+                SCMException.ResultCodes.INTERNAL_ERROR);
+          }
+        }
         response.setAllocateScmBlockResponse(allocateScmBlock(
             request.getAllocateScmBlockRequest(), request.getVersion()));
         break;
@@ -179,9 +191,10 @@
     List<AllocatedBlock> allocatedBlocks =
         impl.allocateBlock(request.getSize(),
             request.getNumBlocks(),
-            ReplicationConfig.fromProtoTypeAndFactor(
+            ReplicationConfig.fromProto(
                 request.getType(),
-                request.getFactor()),
+                request.getFactor(),
+                request.getEcReplicationConfig()),
             request.getOwner(),
             ExcludeList.getFromProtoBuf(request.getExcludeList()));
 
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java
index 26b7c0b..c2ba0ff 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java
@@ -22,6 +22,8 @@
 import com.google.protobuf.ServiceException;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.UpgradeFinalizationStatus;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos;
@@ -99,13 +101,16 @@
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.ha.RatisUtil;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.protocolPB.OzonePBHelper;
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher;
+import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
 import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
+import org.apache.hadoop.ozone.ClientVersion;
 import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -118,6 +123,13 @@
 
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.PipelineResponseProto.Error.errorPipelineAlreadyExists;
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.PipelineResponseProto.Error.success;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetContainer;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetContainerWithPipeline;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetContainerWithPipelineBatch;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetExistContainerWithPipelinesInBatch;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetPipeline;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.ListContainer;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.ListPipelines;
 import static org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol.ADMIN_COMMAND_TYPE;
 
 /**
@@ -133,6 +145,16 @@
   private static final Logger LOG =
       LoggerFactory.getLogger(
           StorageContainerLocationProtocolServerSideTranslatorPB.class);
+  private static final String ERROR_LIST_CONTAINS_EC_REPLICATION_CONFIG =
+      "The returned list of containers contains containers with Erasure Coded"
+          + " replication type, which the client won't be able to understand."
+          + " Please upgrade the client to a version that supports Erasure"
+          + " Coded data, and retry!";
+  private static final String ERROR_RESPONSE_CONTAINS_EC_REPLICATION_CONFIG =
+      "The returned container data contains Erasure Coded replication"
+          + " information, which the client won't be able to understand."
+          + " Please upgrade the client to a version that supports Erasure"
+          + " Coded data, and retry!";
 
   private final StorageContainerLocationProtocol impl;
   private final StorageContainerManager scm;
@@ -170,9 +192,196 @@
           scm.getScmHAManager().getRatisServer().triggerNotLeaderException(),
           scm.getClientRpcPort(), scm.getScmId());
     }
-    return dispatcher
+    // After the request interceptor (now validator) framework is extended to
+    // this server interface, this should be removed and solved via new
+    // annotated interceptors.
+    boolean checkResponseForECRepConfig = false;
+    if (request.getVersion() <
+        ClientVersion.ERASURE_CODING_SUPPORT.toProtoValue()) {
+      if (request.getCmdType() == GetContainer
+          || request.getCmdType() == ListContainer
+          || request.getCmdType() == GetContainerWithPipeline
+          || request.getCmdType() == GetContainerWithPipelineBatch
+          || request.getCmdType() == GetExistContainerWithPipelinesInBatch
+          || request.getCmdType() == ListPipelines
+          || request.getCmdType() == GetPipeline) {
+
+        checkResponseForECRepConfig = true;
+      }
+    }
+    ScmContainerLocationResponse response = dispatcher
         .processRequest(request, this::processRequest, request.getCmdType(),
             request.getTraceID());
+    if (checkResponseForECRepConfig) {
+      try {
+        switch (response.getCmdType()) {
+        case GetContainer:
+          disallowECReplicationConfigInGetContainerResponse(response);
+          break;
+        case ListContainer:
+          disallowECReplicationConfigInListContainerResponse(response);
+          break;
+        case GetContainerWithPipeline:
+          disallowECReplicationConfigInGetContainerWithPipelineResponse(
+              response);
+          break;
+        case GetContainerWithPipelineBatch:
+          disallowECReplicationConfigInGetContainerWithPipelineBatchResponse(
+              response);
+          break;
+        case GetExistContainerWithPipelinesInBatch:
+          disallowECReplicationConfigInGetExistContainerWithPipelineBatchResp(
+              response);
+          break;
+        case ListPipelines:
+          disallowECReplicationConfigInListPipelinesResponse(response);
+          break;
+        case GetPipeline:
+          disallowECReplicationConfigInGetPipelineResponse(response);
+          break;
+        default:
+        }
+      } catch (SCMException e) {
+        throw new ServiceException(e);
+      }
+    }
+    return response;
+  }
+
+  private void disallowECReplicationConfigInListContainerResponse(
+      ScmContainerLocationResponse response) throws SCMException {
+    if (!response.hasScmListContainerResponse()) {
+      return;
+    }
+    for (HddsProtos.ContainerInfoProto containerInfo :
+        response.getScmListContainerResponse().getContainersList()) {
+      if (containerInfo.hasEcReplicationConfig()) {
+        throw new SCMException(ERROR_LIST_CONTAINS_EC_REPLICATION_CONFIG,
+            SCMException.ResultCodes.INTERNAL_ERROR);
+      }
+    }
+  }
+
+  private void disallowECReplicationConfigInGetContainerResponse(
+      ScmContainerLocationResponse response) throws SCMException {
+    if (!response.hasGetContainerResponse()) {
+      return;
+    }
+    if (!response.getGetContainerResponse().hasContainerInfo()) {
+      return;
+    }
+    if (response.getGetContainerResponse().getContainerInfo()
+        .hasEcReplicationConfig()) {
+      throw new SCMException(ERROR_RESPONSE_CONTAINS_EC_REPLICATION_CONFIG,
+          SCMException.ResultCodes.INTERNAL_ERROR);
+    }
+  }
+
+  private void disallowECReplicationConfigInGetContainerWithPipelineResponse(
+      ScmContainerLocationResponse response) throws SCMException {
+    if (!response.hasGetContainerWithPipelineResponse()) {
+      return;
+    }
+    if (!response.getGetContainerWithPipelineResponse()
+        .hasContainerWithPipeline()) {
+      return;
+    }
+    if (response.getGetContainerWithPipelineResponse()
+        .getContainerWithPipeline().hasContainerInfo()) {
+      HddsProtos.ContainerInfoProto containerInfo =
+          response.getGetContainerWithPipelineResponse()
+              .getContainerWithPipeline().getContainerInfo();
+      if (containerInfo.hasEcReplicationConfig()) {
+        throw new SCMException(ERROR_RESPONSE_CONTAINS_EC_REPLICATION_CONFIG,
+            SCMException.ResultCodes.INTERNAL_ERROR);
+      }
+    }
+    if (response.getGetContainerWithPipelineResponse()
+        .getContainerWithPipeline().hasPipeline()) {
+      HddsProtos.Pipeline pipeline =
+          response.getGetContainerWithPipelineResponse()
+              .getContainerWithPipeline().getPipeline();
+      if (pipeline.hasEcReplicationConfig()) {
+        throw new SCMException(ERROR_RESPONSE_CONTAINS_EC_REPLICATION_CONFIG,
+            SCMException.ResultCodes.INTERNAL_ERROR);
+      }
+    }
+  }
+
+  private void
+      disallowECReplicationConfigInGetContainerWithPipelineBatchResponse(
+      ScmContainerLocationResponse response) throws SCMException {
+    if (!response.hasGetContainerWithPipelineBatchResponse()) {
+      return;
+    }
+    List<HddsProtos.ContainerWithPipeline> cwps =
+        response.getGetContainerWithPipelineBatchResponse()
+            .getContainerWithPipelinesList();
+    checkForECReplicationConfigIn(cwps);
+  }
+
+  private void
+      disallowECReplicationConfigInGetExistContainerWithPipelineBatchResp(
+      ScmContainerLocationResponse response) throws SCMException {
+    if (!response.hasGetExistContainerWithPipelinesInBatchResponse()) {
+      return;
+    }
+    List<HddsProtos.ContainerWithPipeline> cwps =
+        response.getGetExistContainerWithPipelinesInBatchResponse()
+            .getContainerWithPipelinesList();
+    checkForECReplicationConfigIn(cwps);
+  }
+
+  private void checkForECReplicationConfigIn(
+      List<HddsProtos.ContainerWithPipeline> cwps)
+      throws SCMException {
+    for (HddsProtos.ContainerWithPipeline cwp : cwps) {
+      if (cwp.hasContainerInfo()) {
+        if (cwp.getContainerInfo().hasEcReplicationConfig()) {
+          throw new SCMException(ERROR_LIST_CONTAINS_EC_REPLICATION_CONFIG,
+              SCMException.ResultCodes.INTERNAL_ERROR);
+        }
+      }
+      if (cwp.hasPipeline()) {
+        if (cwp.getPipeline().hasEcReplicationConfig()) {
+          throw new SCMException(ERROR_LIST_CONTAINS_EC_REPLICATION_CONFIG,
+              SCMException.ResultCodes.INTERNAL_ERROR);
+        }
+      }
+    }
+  }
+
+  private void disallowECReplicationConfigInListPipelinesResponse(
+      ScmContainerLocationResponse response) throws SCMException {
+    if (!response.hasListPipelineResponse()) {
+      return;
+    }
+    for (HddsProtos.Pipeline pipeline :
+        response.getListPipelineResponse().getPipelinesList()) {
+      if (pipeline.hasEcReplicationConfig()) {
+        throw new SCMException("The returned list of pipelines contains"
+            + " pipelines with Erasure Coded replication type, which the"
+            + " client won't be able to understand."
+            + " Please upgrade the client to a version that supports Erasure"
+            + " Coded data, and retry!",
+            SCMException.ResultCodes.INTERNAL_ERROR);
+      }
+    }
+  }
+
+  private void disallowECReplicationConfigInGetPipelineResponse(
+      ScmContainerLocationResponse response) throws SCMException {
+    if (!response.hasGetPipelineResponse()) {
+      return;
+    }
+    if (response.getPipelineResponse().getPipeline().hasEcReplicationConfig()) {
+      throw new SCMException("The returned pipeline data contains"
+          + " Erasure Coded replication information, which the client won't"
+          + " be able to understand."
+          + " Please upgrade the client to a version that supports Erasure"
+          + " Coded data, and retry!",
+          SCMException.ResultCodes.INTERNAL_ERROR);
+    }
   }
 
   @SuppressWarnings("checkstyle:methodlength")
@@ -249,6 +458,18 @@
                 request.getScmCloseContainerRequest()))
             .build();
       case AllocatePipeline:
+        if (scm.getLayoutVersionManager().needsFinalization() &&
+            !scm.getLayoutVersionManager().isAllowed(
+                HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)
+        ) {
+          if (request.getPipelineRequest().getReplicationType() ==
+              HddsProtos.ReplicationType.EC) {
+            throw new SCMException("Cluster is not finalized yet, it is"
+                + " not enabled to create pipelines with Erasure Coded"
+                + " replication type.",
+                SCMException.ResultCodes.INTERNAL_ERROR);
+          }
+        }
         return ScmContainerLocationResponse.newBuilder()
             .setCmdType(request.getCmdType())
             .setStatus(Status.OK)
@@ -528,14 +749,40 @@
     count = request.getCount();
     HddsProtos.LifeCycleState state = null;
     HddsProtos.ReplicationFactor factor = null;
+    HddsProtos.ReplicationType replicationType = null;
+    ReplicationConfig repConfig = null;
     if (request.hasState()) {
       state = request.getState();
     }
-    if (request.hasFactor()) {
+    if (request.hasType()) {
+      replicationType = request.getType();
+    }
+    if (replicationType != null) {
+      // This must come from an upgraded client as the older version never
+      // passed Type. Therefore, we must check for replicationConfig.
+      if (replicationType == HddsProtos.ReplicationType.EC) {
+        if (request.hasEcReplicationConfig()) {
+          repConfig = new ECReplicationConfig(request.getEcReplicationConfig());
+        }
+      } else {
+        if (request.hasFactor()) {
+          repConfig = ReplicationConfig
+              .fromProtoTypeAndFactor(request.getType(), request.getFactor());
+        }
+      }
+    } else if (request.hasFactor()) {
       factor = request.getFactor();
     }
-    List<ContainerInfo> containerList =
-        impl.listContainer(startContainerID, count, state, factor);
+    List<ContainerInfo> containerList;
+    if (factor != null) {
+      // Call from a legacy client
+      containerList =
+          impl.listContainer(startContainerID, count, state, factor);
+    } else {
+      containerList =
+          impl.listContainer(startContainerID, count, state, replicationType,
+              repConfig);
+    }
     SCMListContainerResponseProto.Builder builder =
         SCMListContainerResponseProto.newBuilder();
     for (ContainerInfo container : containerList) {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
index a4d1ae4..3feca15 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
@@ -29,7 +29,6 @@
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
-import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -93,6 +92,7 @@
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StorageContainerLocationProtocolService.newReflectiveBlockingService;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT;
@@ -261,13 +261,9 @@
     }
 
     if (pipeline == null) {
-      pipeline = scm.getPipelineManager().createPipeline(
-          StandaloneReplicationConfig.getInstance(ReplicationConfig
-              .getLegacyFactor(container.getReplicationConfig())),
-          scm.getContainerManager()
-              .getContainerReplicas(cid).stream()
-              .map(ContainerReplica::getDatanodeDetails)
-              .collect(Collectors.toList()));
+      pipeline = scm.getPipelineManager().createPipelineForRead(
+          container.getReplicationConfig(),
+          scm.getContainerManager().getContainerReplicas(cid));
     }
 
     return new ContainerWithPipeline(container, pipeline);
@@ -310,7 +306,8 @@
               .setBytesUsed(r.getBytesUsed())
               .setPlaceOfBirth(r.getOriginDatanodeId().toString())
               .setKeyCount(r.getKeyCount())
-              .setSequenceID(r.getSequenceId()).build()
+              .setSequenceID(r.getSequenceId())
+              .setReplicaIndex(r.getReplicaIndex()).build()
       );
     }
     return results;
@@ -390,7 +387,7 @@
   @Override
   public List<ContainerInfo> listContainer(long startContainerID,
       int count) throws IOException {
-    return listContainer(startContainerID, count, null, null);
+    return listContainer(startContainerID, count, null, null, null);
   }
 
   /**
@@ -406,7 +403,7 @@
   @Override
   public List<ContainerInfo> listContainer(long startContainerID,
       int count, HddsProtos.LifeCycleState state) throws IOException {
-    return listContainer(startContainerID, count, state, null);
+    return listContainer(startContainerID, count, state, null, null);
   }
 
   /**
@@ -420,6 +417,7 @@
    * @throws IOException
    */
   @Override
+  @Deprecated
   public List<ContainerInfo> listContainer(long startContainerID,
       int count, HddsProtos.LifeCycleState state,
       HddsProtos.ReplicationFactor factor) throws IOException {
@@ -439,6 +437,9 @@
         if (factor != null) {
           return scm.getContainerManager().getContainers(state).stream()
               .filter(info -> info.containerID().getId() >= startContainerID)
+              //Filtering EC replication type as EC will not have factor.
+              .filter(info -> info
+                  .getReplicationType() != HddsProtos.ReplicationType.EC)
               .filter(info -> (info.getReplicationFactor() == factor))
               .sorted().limit(count).collect(Collectors.toList());
         } else {
@@ -450,6 +451,9 @@
         if (factor != null) {
           return scm.getContainerManager().getContainers().stream()
               .filter(info -> info.containerID().getId() >= startContainerID)
+              //Filtering EC replication type as EC will not have factor.
+              .filter(info -> info
+                  .getReplicationType() != HddsProtos.ReplicationType.EC)
               .filter(info -> info.getReplicationFactor() == factor)
               .sorted().limit(count).collect(Collectors.toList());
         } else {
@@ -469,6 +473,76 @@
     }
   }
 
+  /**
+   * Lists a range of containers and get their info.
+   *
+   * @param startContainerID start containerID.
+   * @param count count must be {@literal >} 0.
+   * @param state Container with this state will be returned.
+   * @param repConfig Replication Config for the container.
+   * @return a list of pipeline.
+   * @throws IOException
+   */
+  @Override
+  public List<ContainerInfo> listContainer(long startContainerID,
+      int count, HddsProtos.LifeCycleState state,
+      HddsProtos.ReplicationType replicationType,
+      ReplicationConfig repConfig) throws IOException {
+    boolean auditSuccess = true;
+    Map<String, String> auditMap = Maps.newHashMap();
+    auditMap.put("startContainerID", String.valueOf(startContainerID));
+    auditMap.put("count", String.valueOf(count));
+    if (state != null) {
+      auditMap.put("state", state.name());
+    }
+    if (replicationType != null) {
+      auditMap.put("replicationType", replicationType.toString());
+    }
+    if (repConfig != null) {
+      auditMap.put("replicationConfig", repConfig.toString());
+    }
+    try {
+      final ContainerID containerId = ContainerID.valueOf(startContainerID);
+      if (state == null && replicationType == null && repConfig == null) {
+        // Not filters, so just return everything
+        return scm.getContainerManager().getContainers(containerId, count);
+      }
+
+      List<ContainerInfo> containerList;
+      if (state != null) {
+        containerList = scm.getContainerManager().getContainers(state);
+      } else {
+        containerList = scm.getContainerManager().getContainers();
+      }
+
+      Stream<ContainerInfo> containerStream = containerList.stream()
+          .filter(info -> info.containerID().getId() >= startContainerID);
+      // If we have repConfig filter by it, as it includes repType too.
+      // Otherwise, we may have a filter just for repType, eg all EC containers
+      // without filtering on their replication scheme
+      if (repConfig != null) {
+        containerStream = containerStream
+            .filter(info -> info.getReplicationConfig().equals(repConfig));
+      } else if (replicationType != null) {
+        containerStream = containerStream
+            .filter(info -> info.getReplicationType() == replicationType);
+      }
+      return containerStream.sorted()
+          .limit(count)
+          .collect(Collectors.toList());
+    } catch (Exception ex) {
+      auditSuccess = false;
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(SCMAction.LIST_CONTAINER, auditMap, ex));
+      throw ex;
+    } finally {
+      if (auditSuccess) {
+        AUDIT.logReadSuccess(
+            buildAuditMessageForSuccess(SCMAction.LIST_CONTAINER, auditMap));
+      }
+    }
+  }
+
   @Override
   public void deleteContainer(long containerID) throws IOException {
     boolean auditSuccess = true;
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
index 4a82224..c9e4014 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
@@ -19,6 +19,7 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
@@ -651,6 +652,15 @@
         .build();
   }
 
+  public static ContainerInfo getECContainer(
+      final HddsProtos.LifeCycleState state, PipelineID pipelineID,
+      ECReplicationConfig replicationConfig) {
+    return getDefaultContainerInfoBuilder(state)
+        .setReplicationConfig(replicationConfig)
+        .setPipelineID(pipelineID)
+        .build();
+  }
+
   public static Set<ContainerReplica> getReplicas(
       final ContainerID containerId,
       final ContainerReplicaProto.State state,
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java
index d1e97fe..ee0b637 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java
@@ -22,6 +22,7 @@
 
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -169,4 +170,13 @@
     Assert.assertEquals(1, containerManager
         .getContainerStateCount(HddsProtos.LifeCycleState.CLOSED));
   }
+
+  @Test
+  public void testAllocateContainersWithECReplicationConfig() throws Exception {
+    final ContainerInfo admin = containerManager
+        .allocateContainer(new ECReplicationConfig(3, 2), "admin");
+    Assert.assertEquals(1, containerManager.getContainers().size());
+    Assert.assertNotNull(containerManager.getContainer(admin.containerID()));
+  }
+
 }
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
index c653691..c0c8ed2 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
@@ -18,6 +18,7 @@
 
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -53,13 +54,16 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Iterator;
+import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import static junit.framework.TestCase.assertEquals;
+import static org.apache.hadoop.hdds.scm.HddsTestUtils.getECContainer;
 import static org.apache.hadoop.hdds.scm.HddsTestUtils.getReplicas;
 import static org.apache.hadoop.hdds.scm.HddsTestUtils.getContainer;
 
@@ -632,6 +636,158 @@
   }
 
   @Test
+  public void openECContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
+      throws IOException {
+    final ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+    final ContainerReportHandler reportHandler = new ContainerReportHandler(
+        nodeManager, containerManager);
+
+    Pipeline pipeline = pipelineManager.createPipeline(repConfig);
+    Map<Integer, DatanodeDetails> dns = new HashMap<>();
+    final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
+        NodeStatus.inServiceHealthy()).iterator();
+    for (int i = 1; i <= repConfig.getRequiredNodes(); i++) {
+      dns.put(i, nodeIterator.next());
+    }
+    final ContainerReplicaProto.State replicaState
+        = ContainerReplicaProto.State.OPEN;
+    final ContainerInfo containerOne =
+        getECContainer(LifeCycleState.OPEN, pipeline.getId(), repConfig);
+
+    containerStateManager.addContainer(containerOne.getProtobuf());
+    // Container loaded, no replicas reported from DNs. Expect zeros for
+    // usage values.
+    assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Report from data index 2 - should not update stats
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(2), 50L, 60L, 2), publisher);
+    assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Report from replica 1, it should update
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(1), 50L, 60L, 1), publisher);
+    assertEquals(50L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(60L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Parity 1 report a greater value, but as the container is own the stats
+    // should be the min value.
+    // Report from replica 1, it should update
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(4), 80L, 90L, 4), publisher);
+    assertEquals(50L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(60L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Parity 2 reports a lesser value, so the stored values should update
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(5), 40, 30, 5), publisher);
+    assertEquals(40L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(30L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Report from data index 3 - should not update stats even though it has
+    // lesser values
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(2), 10L, 10L, 2), publisher);
+    assertEquals(40L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(30L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+  }
+
+  @Test
+  public void closedECContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
+      throws IOException {
+    final ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+    final ContainerReportHandler reportHandler = new ContainerReportHandler(
+        nodeManager, containerManager);
+
+    Pipeline pipeline = pipelineManager.createPipeline(repConfig);
+    Map<Integer, DatanodeDetails> dns = new HashMap<>();
+    final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
+        NodeStatus.inServiceHealthy()).iterator();
+    for (int i = 1; i <= repConfig.getRequiredNodes(); i++) {
+      dns.put(i, nodeIterator.next());
+    }
+    final ContainerReplicaProto.State replicaState
+        = ContainerReplicaProto.State.OPEN;
+    final ContainerInfo containerOne =
+        getECContainer(LifeCycleState.CLOSED, pipeline.getId(), repConfig);
+
+    containerStateManager.addContainer(containerOne.getProtobuf());
+    // Container loaded, no replicas reported from DNs. Expect zeros for
+    // usage values.
+    assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Report from data index 2 - should not update stats
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(2), 50L, 60L, 2), publisher);
+    assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Report from replica 1, it should update
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(1), 50L, 60L, 1), publisher);
+    assertEquals(50L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(60L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Parity 1 report a greater value, as the container is closed the stats
+    // should be the max value.
+    // Report from replica 1, it should update
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(4), 80L, 90L, 4), publisher);
+    assertEquals(80L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(90L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Parity 2 reports a lesser value, so the stored values should not update
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(5), 40, 30, 5), publisher);
+    assertEquals(80L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(90L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+
+    // Report from data index 3 - should not update stats even though it has
+    // greater values
+    reportHandler.onMessage(getContainerReportFromDatanode(
+        containerOne.containerID(), replicaState,
+        dns.get(2), 110L, 120L, 2), publisher);
+    assertEquals(80L, containerManager.getContainer(containerOne.containerID())
+        .getUsedBytes());
+    assertEquals(90L, containerManager.getContainer(containerOne.containerID())
+        .getNumberOfKeys());
+  }
+
+  @Test
   public void testStaleReplicaOfDeletedContainer() throws NodeNotFoundException,
       IOException {
 
@@ -653,7 +809,7 @@
     // Expects the replica will be deleted.
     final ContainerReportsProto containerReport = getContainerReportsProto(
         containerOne.containerID(), ContainerReplicaProto.State.CLOSED,
-        datanodeOne.getUuidString());
+        datanodeOne.getUuidString(), 0);
     final ContainerReportFromDatanode containerReportFromDatanode =
         new ContainerReportFromDatanode(datanodeOne, containerReport);
     reportHandler.onMessage(containerReportFromDatanode, publisher);
@@ -668,8 +824,16 @@
   private ContainerReportFromDatanode getContainerReportFromDatanode(
       ContainerID containerId, ContainerReplicaProto.State state,
       DatanodeDetails dn, long bytesUsed, long keyCount) {
+    return getContainerReportFromDatanode(containerId, state, dn, bytesUsed,
+        keyCount, 0);
+  }
+
+  private ContainerReportFromDatanode getContainerReportFromDatanode(
+      ContainerID containerId, ContainerReplicaProto.State state,
+      DatanodeDetails dn, long bytesUsed, long keyCount, int replicaIndex) {
     ContainerReportsProto containerReport = getContainerReportsProto(
-        containerId, state, dn.getUuidString(), bytesUsed, keyCount);
+        containerId, state, dn.getUuidString(), bytesUsed, keyCount,
+        replicaIndex);
 
     return new ContainerReportFromDatanode(dn, containerReport);
   }
@@ -678,12 +842,20 @@
       final ContainerID containerId, final ContainerReplicaProto.State state,
       final String originNodeId) {
     return getContainerReportsProto(containerId, state, originNodeId,
-        2000000000L, 100000000L);
+        2000000000L, 100000000L, 0);
   }
 
   protected static ContainerReportsProto getContainerReportsProto(
       final ContainerID containerId, final ContainerReplicaProto.State state,
-      final String originNodeId, final long usedBytes, final long keyCount) {
+      final String originNodeId, int replicaIndex) {
+    return getContainerReportsProto(containerId, state, originNodeId,
+        2000000000L, 100000000L, replicaIndex);
+  }
+
+  protected static ContainerReportsProto getContainerReportsProto(
+      final ContainerID containerId, final ContainerReplicaProto.State state,
+      final String originNodeId, final long usedBytes, final long keyCount,
+      final int replicaIndex) {
     final ContainerReportsProto.Builder crBuilder =
         ContainerReportsProto.newBuilder();
     final ContainerReplicaProto replicaProto =
@@ -701,6 +873,7 @@
             .setWriteBytes(2000000000L)
             .setBlockCommitSequenceId(10000L)
             .setDeleteTransactionId(0)
+            .setReplicaIndex(replicaIndex)
             .build();
     return crBuilder.addReports(replicaProto).build();
   }
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java
index 4ed2887..f85f66e 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java
@@ -165,6 +165,14 @@
     Assert.assertSame(SCMContainerPlacementRandom.class, policy.getClass());
   }
 
+  @Test
+  public void testECPolicy() throws IOException {
+    PlacementPolicy policy = ContainerPlacementPolicyFactory
+        .getECPolicy(conf, null, null, true, null);
+    Assert.assertSame(SCMContainerPlacementRackScatter.class,
+        policy.getClass());
+  }
+
   /**
    * A dummy container placement implementation for test.
    */
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java
index a830a71..12f63fe 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java
@@ -39,6 +39,7 @@
 import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.junit.After;
 import org.apache.hadoop.ozone.container.upgrade.UpgradeUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -196,6 +197,11 @@
         nodeManager, conf, cluster, false, metrics);
   }
 
+  @After
+  public void teardown() {
+    metrics.unRegister();
+  }
+
   @Test
   public void chooseNodeWithNoExcludedNodes() throws SCMException {
     // test choose new datanodes for new pipeline cases
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java
new file mode 100644
index 0000000..1b45a75
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java
@@ -0,0 +1,529 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.container.placement.algorithms;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.MetadataStorageReportProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.HddsTestUtils;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.net.NetConstants;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.net.NetworkTopologyImpl;
+import org.apache.hadoop.hdds.scm.net.Node;
+import org.apache.hadoop.hdds.scm.net.NodeSchema;
+import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
+import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.ozone.container.upgrade.UpgradeUtils;
+import org.hamcrest.MatcherAssert;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test for the scm container rack aware placement.
+ */
+@RunWith(Parameterized.class)
+public class TestSCMContainerPlacementRackScatter {
+  private NetworkTopology cluster;
+  private OzoneConfiguration conf;
+  private NodeManager nodeManager;
+  private final Integer datanodeCount;
+  private final List<DatanodeDetails> datanodes = new ArrayList<>();
+  private final List<DatanodeInfo> dnInfos = new ArrayList<>();
+  // policy with fallback capability
+  private SCMContainerPlacementRackScatter policy;
+  // node storage capacity
+  private static final long STORAGE_CAPACITY = 100L;
+  private SCMContainerPlacementMetrics metrics;
+  private static final int NODE_PER_RACK = 5;
+
+  public TestSCMContainerPlacementRackScatter(Integer count) {
+    this.datanodeCount = count;
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> setupDatanodes() {
+    return Arrays.asList(new Object[][] {{3}, {4}, {5}, {6}, {7}, {8}, {9},
+        {10}, {11}, {12}, {13}, {14}, {15}, {20}, {25}, {30}});
+  }
+
+  @Before
+  public void setup() {
+    //initialize network topology instance
+    conf = new OzoneConfiguration();
+    // We are using small units here
+    conf.setStorageSize(OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN,
+        1, StorageUnit.BYTES);
+    NodeSchema[] schemas = new NodeSchema[]
+        {ROOT_SCHEMA, RACK_SCHEMA, LEAF_SCHEMA};
+    NodeSchemaManager.getInstance().init(schemas, true);
+    cluster = new NetworkTopologyImpl(NodeSchemaManager.getInstance());
+
+    // build datanodes, and network topology
+    String rack = "/rack";
+    String hostname = "node";
+    for (int i = 0; i < datanodeCount; i++) {
+      // Totally 6 racks, each has 5 datanodes
+      DatanodeDetails datanodeDetails =
+          MockDatanodeDetails.createDatanodeDetails(
+          hostname + i, rack + (i / NODE_PER_RACK));
+      DatanodeInfo datanodeInfo = new DatanodeInfo(
+          datanodeDetails, NodeStatus.inServiceHealthy(),
+          UpgradeUtils.defaultLayoutVersionProto());
+
+      StorageReportProto storage1 = HddsTestUtils.createStorageReport(
+          datanodeInfo.getUuid(), "/data1-" + datanodeInfo.getUuidString(),
+          STORAGE_CAPACITY, 0, 100L, null);
+      MetadataStorageReportProto metaStorage1 =
+          HddsTestUtils.createMetadataStorageReport(
+          "/metadata1-" + datanodeInfo.getUuidString(),
+          STORAGE_CAPACITY, 0, 100L, null);
+      datanodeInfo.updateStorageReports(
+          new ArrayList<>(Arrays.asList(storage1)));
+      datanodeInfo.updateMetaDataStorageReports(
+          new ArrayList<>(Arrays.asList(metaStorage1)));
+
+      datanodes.add(datanodeDetails);
+      cluster.add(datanodeDetails);
+      dnInfos.add(datanodeInfo);
+    }
+
+    if (datanodeCount > 4) {
+      StorageReportProto storage2 = HddsTestUtils.createStorageReport(
+          dnInfos.get(2).getUuid(),
+          "/data1-" + datanodes.get(2).getUuidString(),
+          STORAGE_CAPACITY, 90L, 10L, null);
+      dnInfos.get(2).updateStorageReports(
+          new ArrayList<>(Arrays.asList(storage2)));
+      StorageReportProto storage3 = HddsTestUtils.createStorageReport(
+          dnInfos.get(3).getUuid(),
+          "/data1-" + dnInfos.get(3).getUuidString(),
+          STORAGE_CAPACITY, 80L, 20L, null);
+      dnInfos.get(3).updateStorageReports(
+          new ArrayList<>(Arrays.asList(storage3)));
+      StorageReportProto storage4 = HddsTestUtils.createStorageReport(
+          dnInfos.get(4).getUuid(),
+          "/data1-" + dnInfos.get(4).getUuidString(),
+          STORAGE_CAPACITY, 70L, 30L, null);
+      dnInfos.get(4).updateStorageReports(
+          new ArrayList<>(Arrays.asList(storage4)));
+    } else if (datanodeCount > 3) {
+      StorageReportProto storage2 = HddsTestUtils.createStorageReport(
+          dnInfos.get(2).getUuid(),
+          "/data1-" + dnInfos.get(2).getUuidString(),
+          STORAGE_CAPACITY, 90L, 10L, null);
+      dnInfos.get(2).updateStorageReports(
+          new ArrayList<>(Arrays.asList(storage2)));
+      StorageReportProto storage3 = HddsTestUtils.createStorageReport(
+          dnInfos.get(3).getUuid(),
+          "/data1-" + dnInfos.get(3).getUuidString(),
+          STORAGE_CAPACITY, 80L, 20L, null);
+      dnInfos.get(3).updateStorageReports(
+          new ArrayList<>(Arrays.asList(storage3)));
+    } else if (datanodeCount > 2) {
+      StorageReportProto storage2 = HddsTestUtils.createStorageReport(
+          dnInfos.get(2).getUuid(),
+          "/data1-" + dnInfos.get(2).getUuidString(),
+          STORAGE_CAPACITY, 84L, 16L, null);
+      dnInfos.get(2).updateStorageReports(
+          new ArrayList<>(Arrays.asList(storage2)));
+    }
+
+    // create mock node manager
+    nodeManager = Mockito.mock(NodeManager.class);
+    when(nodeManager.getNodes(NodeStatus.inServiceHealthy()))
+        .thenReturn(new ArrayList<>(datanodes));
+    for (DatanodeInfo dn: dnInfos) {
+      when(nodeManager.getNodeByUuid(dn.getUuidString()))
+          .thenReturn(dn);
+    }
+    when(nodeManager.getClusterNetworkTopologyMap())
+        .thenReturn(cluster);
+
+    // create placement policy instances
+    metrics = SCMContainerPlacementMetrics.create();
+    policy = new SCMContainerPlacementRackScatter(
+        nodeManager, conf, cluster, true, metrics);
+  }
+
+  @After
+  public void teardown() {
+    metrics.unRegister();
+  }
+
+  @Test
+  public void chooseNodeWithNoExcludedNodes() throws SCMException {
+    int rackLevel = cluster.getMaxLevel() - 1;
+    int rackNum = cluster.getNumOfNodes(rackLevel);
+
+    // test choose new datanodes for new pipeline cases
+    // 1 replica
+    int nodeNum = 1;
+    List<DatanodeDetails> datanodeDetails =
+        policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+
+    // 2 replicas
+    nodeNum = 2;
+    datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertTrue(!cluster.isSameParent(datanodeDetails.get(0),
+        datanodeDetails.get(1)) || (datanodeCount <= NODE_PER_RACK));
+
+    //  3 replicas
+    nodeNum = 3;
+    if (datanodeCount > nodeNum) {
+      datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+      Assert.assertEquals(nodeNum, datanodeDetails.size());
+      Assert.assertEquals(getRackSize(datanodeDetails),
+          Math.min(nodeNum, rackNum));
+    }
+
+    //  5 replicas
+    nodeNum = 5;
+    if (datanodeCount > nodeNum) {
+      assumeTrue(datanodeCount >= NODE_PER_RACK);
+      datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+      Assert.assertEquals(nodeNum, datanodeDetails.size());
+      Assert.assertEquals(getRackSize(datanodeDetails),
+          Math.min(nodeNum, rackNum));
+    }
+
+    //  10 replicas
+    nodeNum = 10;
+    if (datanodeCount > nodeNum) {
+      assumeTrue(datanodeCount > 2 * NODE_PER_RACK);
+      datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+      Assert.assertEquals(nodeNum, datanodeDetails.size());
+      Assert.assertEquals(getRackSize(datanodeDetails),
+          Math.min(nodeNum, rackNum));
+    }
+  }
+
+  @Test
+  public void chooseNodeWithExcludedNodes() throws SCMException {
+    int rackLevel = cluster.getMaxLevel() - 1;
+    int rackNum = cluster.getNumOfNodes(rackLevel);
+    int totalNum;
+    // test choose new datanodes for under replicated pipeline
+    // 3 replicas, two existing datanodes on same rack
+    assumeTrue(datanodeCount > NODE_PER_RACK);
+    int nodeNum = 1;
+    List<DatanodeDetails> excludedNodes = new ArrayList<>();
+
+    excludedNodes.add(datanodes.get(0));
+    excludedNodes.add(datanodes.get(1));
+    List<DatanodeDetails> datanodeDetails = policy.chooseDatanodes(
+        excludedNodes, null, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertFalse(cluster.isSameParent(datanodeDetails.get(0),
+        excludedNodes.get(0)));
+    Assert.assertFalse(cluster.isSameParent(datanodeDetails.get(0),
+        excludedNodes.get(1)));
+
+    // 3 replicas, one existing datanode
+    nodeNum = 2;
+    totalNum = 3;
+    excludedNodes.clear();
+    excludedNodes.add(datanodes.get(0));
+    datanodeDetails = policy.chooseDatanodes(
+        excludedNodes, null, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
+        Math.min(totalNum, rackNum));
+
+    // 3 replicas, two existing datanodes on different rack
+    nodeNum = 1;
+    totalNum = 3;
+    excludedNodes.clear();
+    excludedNodes.add(datanodes.get(0));
+    excludedNodes.add(datanodes.get(5));
+    datanodeDetails = policy.chooseDatanodes(
+        excludedNodes, null, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
+        Math.min(totalNum, rackNum));
+
+    // 5 replicas, one existing datanode
+    nodeNum = 4;
+    totalNum = 5;
+    excludedNodes.clear();
+    excludedNodes.add(datanodes.get(0));
+    datanodeDetails = policy.chooseDatanodes(
+        excludedNodes, null, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
+        Math.min(totalNum, rackNum));
+
+    // 5 replicas, two existing datanodes on different rack
+    nodeNum = 3;
+    totalNum = 5;
+    excludedNodes.clear();
+    excludedNodes.add(datanodes.get(0));
+    excludedNodes.add(datanodes.get(5));
+    datanodeDetails = policy.chooseDatanodes(
+        excludedNodes, null, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
+        Math.min(totalNum, rackNum));
+  }
+
+  @Test
+  public void chooseNodeWithFavoredNodes() throws SCMException {
+    int nodeNum = 1;
+    List<DatanodeDetails> excludedNodes = new ArrayList<>();
+    List<DatanodeDetails> favoredNodes = new ArrayList<>();
+
+    // no excludedNodes, only favoredNodes
+    favoredNodes.add(datanodes.get(0));
+    List<DatanodeDetails> datanodeDetails = policy.chooseDatanodes(
+        excludedNodes, favoredNodes, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertEquals(datanodeDetails.get(0).getNetworkFullPath(),
+        favoredNodes.get(0).getNetworkFullPath());
+
+    // no overlap between excludedNodes and favoredNodes, favoredNodes can been
+    // chosen.
+    excludedNodes.clear();
+    favoredNodes.clear();
+    excludedNodes.add(datanodes.get(0));
+    favoredNodes.add(datanodes.get(1));
+    datanodeDetails = policy.chooseDatanodes(
+        excludedNodes, favoredNodes, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertEquals(datanodeDetails.get(0).getNetworkFullPath(),
+        favoredNodes.get(0).getNetworkFullPath());
+
+    // there is overlap between excludedNodes and favoredNodes, favoredNodes
+    // should not be chosen.
+    excludedNodes.clear();
+    favoredNodes.clear();
+    excludedNodes.add(datanodes.get(0));
+    favoredNodes.add(datanodes.get(0));
+    datanodeDetails = policy.chooseDatanodes(
+        excludedNodes, favoredNodes, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertFalse(datanodeDetails.get(0).getNetworkFullPath()
+        .equals(favoredNodes.get(0).getNetworkFullPath()));
+  }
+
+  @Test
+  public void testNoInfiniteLoop() throws SCMException {
+    int nodeNum = 1;
+
+    try {
+      // request storage space larger than node capability
+      policy.chooseDatanodes(null, null, nodeNum, STORAGE_CAPACITY + 0, 15);
+      fail("Storage requested exceeds capacity, this call should fail");
+    } catch (Exception e) {
+      assertTrue(e.getClass().getSimpleName().equals("SCMException"));
+    }
+
+    // get metrics
+    long totalRequest = metrics.getDatanodeRequestCount();
+    long successCount = metrics.getDatanodeChooseSuccessCount();
+    long tryCount = metrics.getDatanodeChooseAttemptCount();
+    long compromiseCount = metrics.getDatanodeChooseFallbackCount();
+
+    Assert.assertEquals(totalRequest, nodeNum);
+    Assert.assertEquals(successCount, 0);
+    MatcherAssert.assertThat("Not enough try", tryCount,
+        greaterThanOrEqualTo((long) nodeNum));
+    Assert.assertEquals(compromiseCount, 0);
+  }
+
+  @Test
+  public void testDatanodeWithDefaultNetworkLocation() throws SCMException {
+    String hostname = "node";
+    List<DatanodeInfo> dnInfoList = new ArrayList<>();
+    List<DatanodeDetails> dataList = new ArrayList<>();
+    NetworkTopology clusterMap =
+        new NetworkTopologyImpl(NodeSchemaManager.getInstance());
+    for (int i = 0; i < 30; i++) {
+      // Totally 6 racks, each has 5 datanodes
+      DatanodeDetails dn = MockDatanodeDetails.createDatanodeDetails(
+          hostname + i, null);
+      DatanodeInfo dnInfo = new DatanodeInfo(
+          dn, NodeStatus.inServiceHealthy(),
+          UpgradeUtils.defaultLayoutVersionProto());
+
+      StorageReportProto storage1 = HddsTestUtils.createStorageReport(
+          dnInfo.getUuid(), "/data1-" + dnInfo.getUuidString(),
+          STORAGE_CAPACITY, 0, 100L, null);
+      MetadataStorageReportProto metaStorage1 =
+          HddsTestUtils.createMetadataStorageReport(
+          "/metadata1-" + dnInfo.getUuidString(),
+          STORAGE_CAPACITY, 0, 100L, null);
+      dnInfo.updateStorageReports(
+          new ArrayList<>(Arrays.asList(storage1)));
+      dnInfo.updateMetaDataStorageReports(
+          new ArrayList<>(Arrays.asList(metaStorage1)));
+
+      dataList.add(dn);
+      clusterMap.add(dn);
+      dnInfoList.add(dnInfo);
+    }
+    Assert.assertEquals(dataList.size(), StringUtils.countMatches(
+        clusterMap.toString(), NetConstants.DEFAULT_RACK));
+    for (DatanodeInfo dn: dnInfoList) {
+      when(nodeManager.getNodeByUuid(dn.getUuidString()))
+          .thenReturn(dn);
+    }
+
+    // choose nodes to host 5 replica
+    int nodeNum = 5;
+    SCMContainerPlacementRackScatter newPolicy =
+        new SCMContainerPlacementRackScatter(nodeManager, conf, clusterMap,
+            true, metrics);
+    List<DatanodeDetails> datanodeDetails =
+        newPolicy.chooseDatanodes(null, null, nodeNum, 0, 15);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+    Assert.assertEquals(1, getRackSize(datanodeDetails));
+  }
+
+  @Test
+  public void testvalidateContainerPlacement() {
+    // Only run this test for the full set of DNs. 5 DNs per rack on 6 racks.
+    assumeTrue(datanodeCount >= 15);
+    List<DatanodeDetails> dns = new ArrayList<>();
+    // First 5 node are on the same rack
+    dns.add(datanodes.get(0));
+    dns.add(datanodes.get(1));
+    dns.add(datanodes.get(2));
+    ContainerPlacementStatus stat = policy.validateContainerPlacement(dns, 3);
+    assertFalse(stat.isPolicySatisfied());
+    assertEquals(2, stat.misReplicationCount());
+
+    // Pick a new list which spans 2 racks
+    dns = new ArrayList<>();
+    dns.add(datanodes.get(0));
+    dns.add(datanodes.get(1));
+    dns.add(datanodes.get(5)); // This is on second rack
+    stat = policy.validateContainerPlacement(dns, 3);
+    assertFalse(stat.isPolicySatisfied());
+    assertEquals(1, stat.misReplicationCount());
+
+    // Pick single DN, expecting 3 replica. Policy is not met.
+    dns = new ArrayList<>();
+    dns.add(datanodes.get(0));
+    stat = policy.validateContainerPlacement(dns, 3);
+    assertFalse(stat.isPolicySatisfied());
+    assertEquals(2, stat.misReplicationCount());
+
+    // Pick single DN, expecting 1 replica. Policy is met.
+    dns = new ArrayList<>();
+    dns.add(datanodes.get(0));
+    stat = policy.validateContainerPlacement(dns, 1);
+    assertTrue(stat.isPolicySatisfied());
+    assertEquals(0, stat.misReplicationCount());
+  }
+
+  @Test
+  public void testvalidateContainerPlacementSingleRackCluster() {
+    assumeTrue(datanodeCount == 5);
+
+    // All nodes are on the same rack in this test, and the cluster only has
+    // one rack.
+    List<DatanodeDetails> dns = new ArrayList<>();
+    dns.add(datanodes.get(0));
+    dns.add(datanodes.get(1));
+    dns.add(datanodes.get(2));
+    ContainerPlacementStatus stat = policy.validateContainerPlacement(dns, 3);
+    assertTrue(stat.isPolicySatisfied());
+    assertEquals(0, stat.misReplicationCount());
+
+    // Single DN - policy met as cluster only has one rack.
+    dns = new ArrayList<>();
+    dns.add(datanodes.get(0));
+    stat = policy.validateContainerPlacement(dns, 3);
+    assertTrue(stat.isPolicySatisfied());
+    assertEquals(0, stat.misReplicationCount());
+
+    // Single DN - only 1 replica expected
+    dns = new ArrayList<>();
+    dns.add(datanodes.get(0));
+    stat = policy.validateContainerPlacement(dns, 1);
+    assertTrue(stat.isPolicySatisfied());
+    assertEquals(0, stat.misReplicationCount());
+  }
+
+  @Test
+  public void testExcludedNodesOverlapsOutOfServiceNodes() throws SCMException {
+    assumeTrue(datanodeCount == 6);
+
+    // DN 5 is out of service
+    dnInfos.get(5).setNodeStatus(new NodeStatus(DECOMMISSIONED, HEALTHY));
+
+    // SCM should have detected that DN 5 is dead
+    cluster.remove(datanodes.get(5));
+
+    // Here we still have 5 DNs, so pick 5 should be possible
+    int nodeNum = 5;
+    List<DatanodeDetails> excludedNodes = new ArrayList<>();
+    // The DN 5 is out of service,
+    // but the client already has it in the excludeList.
+    // So there is an overlap.
+    excludedNodes.add(datanodes.get(5));
+
+    List<DatanodeDetails> datanodeDetails = policy.chooseDatanodes(
+        excludedNodes, null, nodeNum, 0, 5);
+    Assert.assertEquals(nodeNum, datanodeDetails.size());
+  }
+
+  private int getRackSize(List<DatanodeDetails>... datanodeDetails) {
+    Set<Node> racks = new HashSet<>();
+    for (List<DatanodeDetails> list : datanodeDetails) {
+      for (DatanodeDetails dn : list) {
+        racks.add(cluster.getAncestor(dn, 1));
+      }
+    }
+    return racks.size();
+  }
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java
index c82006e..523efc0 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java
@@ -20,7 +20,9 @@
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
 import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
@@ -29,17 +31,21 @@
 import org.apache.hadoop.ozone.ClientVersion;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
+import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 /**
  * Mock PipelineManager implementation for testing.
  */
-public final class MockPipelineManager implements PipelineManager {
+public class MockPipelineManager implements PipelineManager {
 
   private PipelineStateManager stateManager;
 
@@ -56,6 +62,14 @@
   @Override
   public Pipeline createPipeline(ReplicationConfig replicationConfig)
       throws IOException {
+    return createPipeline(replicationConfig, Collections.emptyList(),
+        Collections.emptyList());
+  }
+
+  @Override
+  public Pipeline createPipeline(ReplicationConfig replicationConfig,
+      List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+      throws IOException {
     final List<DatanodeDetails> nodes = Stream.generate(
         MockDatanodeDetails::randomDatanodeDetails)
         .limit(replicationConfig.getRequiredNodes())
@@ -84,6 +98,25 @@
   }
 
   @Override
+  public Pipeline createPipelineForRead(
+      final ReplicationConfig replicationConfig,
+      final Set<ContainerReplica> replicas) {
+    List<DatanodeDetails> dns = new ArrayList<>();
+    Map<DatanodeDetails, Integer> map = new HashMap<>();
+    for (ContainerReplica r : replicas) {
+      map.put(r.getDatanodeDetails(), r.getReplicaIndex());
+      dns.add(r.getDatanodeDetails());
+    }
+    return Pipeline.newBuilder()
+        .setId(PipelineID.randomId())
+        .setReplicationConfig(replicationConfig)
+        .setNodes(dns)
+        .setReplicaIndexes(map)
+        .setState(Pipeline.PipelineState.CLOSED)
+        .build();
+  }
+
+  @Override
   public Pipeline getPipeline(final PipelineID pipelineID)
       throws PipelineNotFoundException {
     return stateManager.getPipeline(pipelineID);
@@ -126,6 +159,19 @@
   }
 
   @Override
+  /**
+   * Returns the count of pipelines meeting the given ReplicationConfig and
+   * state.
+   * @param replicationConfig The ReplicationConfig of the pipelines to count
+   * @param state The current state of the pipelines to count
+   * @return The count of pipelines meeting the above criteria
+   */
+  public int getPipelineCount(ReplicationConfig replicationConfig,
+      final Pipeline.PipelineState state) {
+    return stateManager.getPipelineCount(replicationConfig, state);
+  }
+
+  @Override
   public void addContainerToPipeline(final PipelineID pipelineID,
                                      final ContainerID containerID)
       throws IOException {
@@ -160,11 +206,15 @@
   @Override
   public void openPipeline(final PipelineID pipelineId)
       throws IOException {
+    stateManager.updatePipelineState(
+        pipelineId.getProtobuf(), HddsProtos.PipelineState.PIPELINE_OPEN);
   }
 
   @Override
   public void closePipeline(final Pipeline pipeline, final boolean onTimeout)
       throws IOException {
+    stateManager.updatePipelineState(pipeline.getId().getProtobuf(),
+        HddsProtos.PipelineState.PIPELINE_CLOSED);
   }
 
   @Override
@@ -206,6 +256,8 @@
   @Override
   public void deactivatePipeline(final PipelineID pipelineID)
       throws IOException {
+    stateManager.updatePipelineState(pipelineID.getProtobuf(),
+        HddsProtos.PipelineState.PIPELINE_DORMANT);
   }
 
   @Override
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestECPipelineProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestECPipelineProvider.java
new file mode 100644
index 0000000..802ed01
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestECPipelineProvider.java
@@ -0,0 +1,154 @@
+/*
+ * 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.hdds.scm.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.ALLOCATED;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test for the ECPipelineProvider.
+ */
+public class TestECPipelineProvider {
+
+  private PipelineProvider provider;
+  private OzoneConfiguration conf;
+  private NodeManager nodeManager = Mockito.mock(NodeManager.class);
+  private PipelineStateManager stateManager =
+      Mockito.mock(PipelineStateManager.class);
+  private PlacementPolicy placementPolicy = Mockito.mock(PlacementPolicy.class);
+  private long containerSizeBytes;
+  @Before
+  public void setup() throws IOException {
+    conf = new OzoneConfiguration();
+    provider = new ECPipelineProvider(
+        nodeManager, stateManager, conf, placementPolicy);
+    this.containerSizeBytes = (long) this.conf.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+        StorageUnit.BYTES);
+    // Placement policy will always return EC number of random nodes.
+    Mockito.when(placementPolicy.chooseDatanodes(Mockito.anyList(),
+        Mockito.anyList(), Mockito.anyInt(), Mockito.anyLong(),
+        Mockito.anyLong()))
+        .thenAnswer(invocation -> {
+          List<DatanodeDetails> dns = new ArrayList<>();
+          for (int i = 0; i < (int) invocation.getArguments()[2]; i++) {
+            dns.add(MockDatanodeDetails.randomDatanodeDetails());
+          }
+          return dns;
+        });
+
+  }
+
+
+  @Test
+  public void testSimplePipelineCanBeCreatedWithIndexes() throws IOException {
+    ECReplicationConfig ecConf = new ECReplicationConfig(3, 2);
+    Pipeline pipeline = provider.create(ecConf);
+    Assert.assertEquals(EC, pipeline.getType());
+    Assert.assertEquals(ecConf.getData() + ecConf.getParity(),
+        pipeline.getNodes().size());
+    Assert.assertEquals(ALLOCATED, pipeline.getPipelineState());
+    List<DatanodeDetails> dns = pipeline.getNodes();
+    for (int i = 0; i < ecConf.getRequiredNodes(); i++) {
+      // EC DN indexes are numbered starting from 1 to N.
+      Assert.assertEquals(i + 1, pipeline.getReplicaIndex(dns.get(i)));
+    }
+  }
+
+  @Test
+  public void testPipelineForReadCanBeCreated() {
+    ECReplicationConfig ecConf = new ECReplicationConfig(3, 2);
+
+    Set<ContainerReplica> replicas = createContainerReplicas(4);
+    Pipeline pipeline = provider.createForRead(ecConf, replicas);
+
+    Assert.assertEquals(EC, pipeline.getType());
+    Assert.assertEquals(4, pipeline.getNodes().size());
+    Assert.assertEquals(ALLOCATED, pipeline.getPipelineState());
+    for (ContainerReplica r : replicas) {
+      Assert.assertEquals(r.getReplicaIndex(),
+          pipeline.getReplicaIndex(r.getDatanodeDetails()));
+    }
+  }
+
+  @Test
+  public void testExcludedAndFavoredNodesPassedToPlacementPolicy()
+      throws IOException {
+    ECReplicationConfig ecConf = new ECReplicationConfig(3, 2);
+
+    List<DatanodeDetails> excludedNodes = new ArrayList<>();
+    excludedNodes.add(MockDatanodeDetails.randomDatanodeDetails());
+
+    List<DatanodeDetails> favoredNodes = new ArrayList<>();
+    favoredNodes.add(MockDatanodeDetails.randomDatanodeDetails());
+
+    Pipeline pipeline = provider.create(ecConf, excludedNodes, favoredNodes);
+    Assert.assertEquals(EC, pipeline.getType());
+    Assert.assertEquals(ecConf.getData() + ecConf.getParity(),
+        pipeline.getNodes().size());
+
+    verify(placementPolicy).chooseDatanodes(excludedNodes, favoredNodes,
+        ecConf.getRequiredNodes(), 0, containerSizeBytes);
+  }
+
+  private Set<ContainerReplica> createContainerReplicas(int number) {
+    Set<ContainerReplica> replicas = new HashSet<>();
+    for (int i = 0; i < number; i++) {
+      ContainerReplica r = ContainerReplica.newBuilder()
+          .setBytesUsed(1)
+          .setContainerID(ContainerID.valueOf(1))
+          .setContainerState(StorageContainerDatanodeProtocolProtos
+              .ContainerReplicaProto.State.CLOSED)
+          .setKeyCount(1)
+          .setOriginNodeId(UUID.randomUUID())
+          .setSequenceId(1)
+          .setReplicaIndex(i + 1)
+          .setDatanodeDetails(MockDatanodeDetails.randomDatanodeDetails())
+          .build();
+      replicas.add(r);
+    }
+    return replicas;
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java
index 3a68db1..fd29929 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java
@@ -25,10 +25,12 @@
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.HddsTestUtils;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.ha.SCMHADBTransactionBuffer;
@@ -37,6 +39,7 @@
 import org.apache.hadoop.hdds.scm.ha.SCMContext;
 import org.apache.hadoop.hdds.scm.ha.SCMServiceManager;
 import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
 import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
@@ -57,9 +60,12 @@
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT_DEFAULT;
@@ -223,12 +229,18 @@
         .getPipelines(RatisReplicationConfig
             .getInstance(ReplicationFactor.THREE),
             Pipeline.PipelineState.OPEN).contains(pipeline));
+    Assert.assertEquals(1, pipelineManager.getPipelineCount(
+        RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
+            Pipeline.PipelineState.DORMANT));
 
     pipelineManager.activatePipeline(pipeline.getId());
     Assert.assertTrue(pipelineManager
         .getPipelines(RatisReplicationConfig
             .getInstance(ReplicationFactor.THREE),
             Pipeline.PipelineState.OPEN).contains(pipeline));
+    Assert.assertEquals(1, pipelineManager.getPipelineCount(
+        RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
+            Pipeline.PipelineState.OPEN));
     buffer.flush();
     Assert.assertTrue(pipelineStore.get(pipeline.getId()).isOpen());
     pipelineManager.close();
@@ -724,6 +736,43 @@
     assertTrue(containerLogIdx < pipelineLogIdx);
   }
 
+  @Test
+  public void testCreatePipelineForRead() throws IOException {
+    PipelineManager pipelineManager = createPipelineManager(true);
+    List<DatanodeDetails> dns = nodeManager
+        .getNodes(NodeStatus.inServiceHealthy())
+        .stream()
+        .limit(3)
+        .collect(Collectors.toList());
+    Set<ContainerReplica> replicas = createContainerReplicasList(dns);
+    Pipeline pipeline = pipelineManager.createPipelineForRead(
+        RatisReplicationConfig.getInstance(ReplicationFactor.THREE), replicas);
+    Assert.assertEquals(3, pipeline.getNodes().size());
+    for (DatanodeDetails dn : pipeline.getNodes())  {
+      Assert.assertTrue(dns.contains(dn));
+    }
+  }
+
+  private Set<ContainerReplica> createContainerReplicasList(
+      List <DatanodeDetails> dns) {
+    Set<ContainerReplica> replicas = new HashSet<>();
+    for (DatanodeDetails dn : dns) {
+      ContainerReplica r = ContainerReplica.newBuilder()
+          .setBytesUsed(1)
+          .setContainerID(ContainerID.valueOf(1))
+          .setContainerState(StorageContainerDatanodeProtocolProtos
+              .ContainerReplicaProto.State.CLOSED)
+          .setKeyCount(1)
+          .setOriginNodeId(UUID.randomUUID())
+          .setSequenceId(1)
+          .setReplicaIndex(0)
+          .setDatanodeDetails(dn)
+          .build();
+      replicas.add(r);
+    }
+    return replicas;
+  }
+
   private void sendPipelineReport(
       DatanodeDetails dn, Pipeline pipeline,
       PipelineReportHandler pipelineReportHandler,
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateMap.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateMap.java
new file mode 100644
index 0000000..8168ce8
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateMap.java
@@ -0,0 +1,99 @@
+/**
+ * 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.hdds.scm.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for PipelineStateMap.
+ */
+
+public class TestPipelineStateMap {
+
+  private PipelineStateMap map;
+
+  @Before
+  public void setup() {
+    map = new PipelineStateMap();
+  }
+
+  @After
+  public void teardown() throws IOException {
+  }
+
+  @Test
+  public void testCountPipelines() throws IOException {
+    Pipeline p;
+
+    // Open Stanadlone Pipelines
+    map.addPipeline(MockPipeline.createPipeline(1));
+    map.addPipeline(MockPipeline.createPipeline(1));
+    p = MockPipeline.createPipeline(1);
+    map.addPipeline(p);
+    map.updatePipelineState(p.getId(), Pipeline.PipelineState.CLOSED);
+
+    // Ratis pipeline
+    map.addPipeline(MockPipeline.createRatisPipeline());
+    p = MockPipeline.createRatisPipeline();
+    map.addPipeline(p);
+    map.updatePipelineState(p.getId(), Pipeline.PipelineState.CLOSED);
+
+    // EC Pipelines
+    map.addPipeline(MockPipeline.createEcPipeline(
+        new ECReplicationConfig(3, 2)));
+    map.addPipeline(MockPipeline.createEcPipeline(
+        new ECReplicationConfig(3, 2)));
+    p = MockPipeline.createEcPipeline(new ECReplicationConfig(3, 2));
+    map.addPipeline(p);
+    map.updatePipelineState(p.getId(), Pipeline.PipelineState.CLOSED);
+
+    assertEquals(2, map.getPipelineCount(
+        StandaloneReplicationConfig.getInstance(ONE),
+        Pipeline.PipelineState.OPEN));
+    assertEquals(1, map.getPipelineCount(
+        RatisReplicationConfig.getInstance(THREE),
+        Pipeline.PipelineState.OPEN));
+    assertEquals(2, map.getPipelineCount(new ECReplicationConfig(3, 2),
+        Pipeline.PipelineState.OPEN));
+
+    assertEquals(0, map.getPipelineCount(new ECReplicationConfig(6, 3),
+        Pipeline.PipelineState.OPEN));
+
+    assertEquals(1, map.getPipelineCount(
+        StandaloneReplicationConfig.getInstance(ONE),
+        Pipeline.PipelineState.CLOSED));
+    assertEquals(1, map.getPipelineCount(
+        RatisReplicationConfig.getInstance(THREE),
+        Pipeline.PipelineState.CLOSED));
+    assertEquals(1, map.getPipelineCount(new ECReplicationConfig(3, 2),
+        Pipeline.PipelineState.CLOSED));
+  }
+
+
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
index d2b19fb..c0ce8d4 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
@@ -27,7 +27,10 @@
 import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
 import org.apache.hadoop.hdds.scm.container.TestContainerManagerImpl;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
@@ -46,6 +49,7 @@
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -56,6 +60,7 @@
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -219,6 +224,7 @@
     List<DatanodeDetails> healthyNodes = nodeManager
         .getNodes(NodeStatus.inServiceHealthy()).stream()
         .limit(3).collect(Collectors.toList());
+    Set<ContainerReplica> replicas = createContainerReplicas(healthyNodes);
 
     Pipeline pipeline1 = provider.create(
         RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
@@ -226,8 +232,12 @@
     Pipeline pipeline2 = provider.create(
         RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
         healthyNodes);
+    Pipeline pipeline3 = provider.createForRead(
+        RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
+        replicas);
 
     Assert.assertEquals(pipeline1.getNodeSet(), pipeline2.getNodeSet());
+    Assert.assertEquals(pipeline2.getNodeSet(), pipeline3.getNodeSet());
     cleanup();
   }
 
@@ -281,6 +291,31 @@
   }
 
   @Test
+  // Test excluded nodes work correctly. Note that for Ratis, the
+  // PipelinePlacementPolicy, which Ratis is hardcoded to use, does not consider
+  // favored nodes.
+  public void testCreateFactorTHREEPipelineWithExcludedDatanodes()
+      throws Exception {
+    init(1);
+    int healthyCount = nodeManager.getNodes(NodeStatus.inServiceHealthy())
+        .size();
+    // Add all but 3 nodes to the exclude list and ensure that the 3 picked
+    // nodes are not in the excluded list.
+    List<DatanodeDetails> excludedNodes = nodeManager
+        .getNodes(NodeStatus.inServiceHealthy()).stream()
+        .limit(healthyCount - 3).collect(Collectors.toList());
+
+    Pipeline pipeline1 = provider.create(
+        RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
+        excludedNodes, Collections.EMPTY_LIST);
+
+    for (DatanodeDetails dn : pipeline1.getNodes()) {
+      assertFalse(excludedNodes.contains(dn));
+    }
+  }
+
+
+  @Test
   public void testCreatePipelinesWhenNotEnoughSpace() throws Exception {
     String expectedErrorSubstring = "Unable to find enough" +
         " nodes that meet the space requirement";
@@ -291,6 +326,9 @@
     largeContainerConf.set(OZONE_SCM_CONTAINER_SIZE, "100TB");
     init(1, largeContainerConf);
     for (ReplicationFactor factor: ReplicationFactor.values()) {
+      if (factor == ReplicationFactor.ZERO) {
+        continue;
+      }
       try {
         provider.create(RatisReplicationConfig.getInstance(factor));
         Assert.fail("Expected SCMException for large container size with " +
@@ -304,6 +342,9 @@
     largeMetadataConf.set(OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN, "100TB");
     init(1, largeMetadataConf);
     for (ReplicationFactor factor: ReplicationFactor.values()) {
+      if (factor == ReplicationFactor.ZERO) {
+        continue;
+      }
       try {
         provider.create(RatisReplicationConfig.getInstance(factor));
         Assert.fail("Expected SCMException for large metadata size with " +
@@ -331,4 +372,24 @@
     stateManager.addPipeline(pipelineProto);
     nodeManager.addPipeline(openPipeline);
   }
+
+  private Set<ContainerReplica> createContainerReplicas(
+      List<DatanodeDetails> dns) {
+    Set<ContainerReplica> replicas = new HashSet<>();
+    for (DatanodeDetails dn : dns) {
+      ContainerReplica r = ContainerReplica.newBuilder()
+          .setBytesUsed(1)
+          .setContainerID(ContainerID.valueOf(1))
+          .setContainerState(StorageContainerDatanodeProtocolProtos
+              .ContainerReplicaProto.State.CLOSED)
+          .setKeyCount(1)
+          .setOriginNodeId(UUID.randomUUID())
+          .setSequenceId(1)
+          .setReplicaIndex(0)
+          .setDatanodeDetails(dn)
+          .build();
+      replicas.add(r);
+    }
+    return replicas;
+  }
 }
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java
new file mode 100644
index 0000000..ae997a3
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java
@@ -0,0 +1,446 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.pipeline;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.PipelineChoosePolicy;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.TestContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.HealthyPipelineChoosePolicy;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdds.conf.StorageUnit.BYTES;
+import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.CLOSED;
+import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.OPEN;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests to validate the WritableECContainerProvider works correctly.
+ */
+public class TestWritableECContainerProvider {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestWritableECContainerProvider.class);
+  private static final String OWNER = "SCM";
+  private PipelineManager pipelineManager;
+  private ContainerManager containerManager
+      = Mockito.mock(ContainerManager.class);
+  private PipelineChoosePolicy pipelineChoosingPolicy
+      = new HealthyPipelineChoosePolicy();
+
+  private OzoneConfiguration conf;
+  private DBStore dbStore;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private WritableContainerProvider provider;
+  private ReplicationConfig repConfig;
+  private int minPipelines;
+
+  private Map<ContainerID, ContainerInfo> containers;
+
+  @Before
+  public void setup() throws IOException {
+    repConfig = new ECReplicationConfig(3, 2);
+    conf = new OzoneConfiguration();
+    WritableECContainerProvider.WritableECContainerProviderConfig providerConf =
+        conf.getObject(WritableECContainerProvider
+            .WritableECContainerProviderConfig.class);
+    minPipelines = providerConf.getMinimumPipelines();
+    containers = new HashMap<>();
+    File testDir = GenericTestUtils.getTestDir(
+        TestContainerManagerImpl.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+    dbStore = DBStoreBuilder.createDBStore(
+        conf, new SCMDBDefinition());
+    scmhaManager = SCMHAManagerStub.getInstance(true);
+    nodeManager = new MockNodeManager(true, 10);
+    pipelineManager =
+        new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+    provider = new WritableECContainerProvider(
+        conf, pipelineManager, containerManager, pipelineChoosingPolicy);
+
+    Mockito.doAnswer(call -> {
+      Pipeline pipeline = (Pipeline)call.getArguments()[2];
+      ContainerInfo container = createContainer(pipeline,
+          repConfig, System.nanoTime());
+      pipelineManager.addContainerToPipeline(
+          pipeline.getId(), container.containerID());
+      containers.put(container.containerID(), container);
+      return container;
+    }).when(containerManager).getMatchingContainer(Matchers.anyLong(),
+        Matchers.anyString(), Matchers.any(Pipeline.class));
+
+    Mockito.doAnswer(call ->
+        containers.get((ContainerID)call.getArguments()[0]))
+        .when(containerManager).getContainer(Matchers.any(ContainerID.class));
+
+  }
+
+  @Test
+  public void testPipelinesCreatedUpToMinLimitAndRandomPipelineReturned()
+      throws IOException {
+    // The first 5 calls should return a different container
+    Set<ContainerInfo> allocatedContainers = new HashSet<>();
+    for (int i = 0; i < minPipelines; i++) {
+      ContainerInfo container =
+          provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+      assertFalse(allocatedContainers.contains(container));
+      allocatedContainers.add(container);
+    }
+
+    allocatedContainers.clear();
+    for (int i = 0; i < 20; i++) {
+      ContainerInfo container =
+          provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+      allocatedContainers.add(container);
+    }
+    // Should have minPipelines containers created
+    assertEquals(minPipelines,
+        pipelineManager.getPipelines(repConfig, OPEN).size());
+    // We should have more than 1 allocatedContainers in the set proving a
+    // random container is selected each time. Do not check for 5 here as there
+    // is a reasonable chance that in 20 turns we don't pick all 5 nodes.
+    assertTrue(allocatedContainers.size() > 2);
+  }
+
+  @Test
+  public void testPiplineLimitIgnoresExcludedPipelines() throws IOException {
+    Set<ContainerInfo> allocatedContainers = new HashSet<>();
+    for (int i = 0; i < minPipelines; i++) {
+      ContainerInfo container = provider.getContainer(
+          1, repConfig, OWNER, new ExcludeList());
+      allocatedContainers.add(container);
+    }
+    // We have the min limit of pipelines, but then exclude one. It should use
+    // one of the existing rather than createing a new one, as the limit is
+    // checked against all pipelines, not just the filtered list
+    ExcludeList exclude = new ExcludeList();
+    PipelineID excludedID = allocatedContainers
+        .stream().findFirst().get().getPipelineID();
+    exclude.addPipeline(excludedID);
+
+    ContainerInfo c = provider.getContainer(1, repConfig, OWNER, exclude);
+    assertNotEquals(excludedID, c.getPipelineID());
+    assertTrue(allocatedContainers.contains(c));
+  }
+
+  @Test
+  public void testNewPipelineCreatedIfAllPipelinesExcluded()
+      throws IOException {
+    Set<ContainerInfo> allocatedContainers = new HashSet<>();
+    for (int i = 0; i < minPipelines; i++) {
+      ContainerInfo container = provider.getContainer(
+          1, repConfig, OWNER, new ExcludeList());
+      allocatedContainers.add(container);
+    }
+    // We have the min limit of pipelines, but then exclude one. It should use
+    // one of the existing rather than creating a new one, as the limit is
+    // checked against all pipelines, not just the filtered list
+    ExcludeList exclude = new ExcludeList();
+    for (ContainerInfo c : allocatedContainers) {
+      exclude.addPipeline(c.getPipelineID());
+    }
+    ContainerInfo c = provider.getContainer(1, repConfig, OWNER, exclude);
+    assertFalse(allocatedContainers.contains(c));
+  }
+
+  @Test
+  public void testNewPipelineCreatedIfAllContainersExcluded()
+      throws IOException {
+    Set<ContainerInfo> allocatedContainers = new HashSet<>();
+    for (int i = 0; i < minPipelines; i++) {
+      ContainerInfo container = provider.getContainer(
+          1, repConfig, OWNER, new ExcludeList());
+      allocatedContainers.add(container);
+    }
+    // We have the min limit of pipelines, but then exclude one. It should use
+    // one of the existing rather than createing a new one, as the limit is
+    // checked against all pipelines, not just the filtered list
+    ExcludeList exclude = new ExcludeList();
+    for (ContainerInfo c : allocatedContainers) {
+      exclude.addConatinerId(c.containerID());
+    }
+    ContainerInfo c = provider.getContainer(1, repConfig, OWNER, exclude);
+    assertFalse(allocatedContainers.contains(c));
+  }
+
+  @Test
+  public void testUnableToCreateAnyPipelinesReturnsNull() throws IOException {
+    pipelineManager = new MockPipelineManager(
+        dbStore, scmhaManager, nodeManager) {
+      @Override
+      public Pipeline createPipeline(ReplicationConfig repConf,
+          List<DatanodeDetails> excludedNodes,
+          List<DatanodeDetails> favoredNodes) throws IOException {
+        throw new IOException("Cannot create pipelines");
+      }
+    };
+    provider = new WritableECContainerProvider(
+        conf, pipelineManager, containerManager, pipelineChoosingPolicy);
+
+    ContainerInfo container =
+        provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+    assertNull(container);
+  }
+
+  @Test
+  public void testExistingPipelineReturnedWhenNewCannotBeCreated()
+      throws IOException {
+    pipelineManager = new MockPipelineManager(
+        dbStore, scmhaManager, nodeManager) {
+
+      private boolean throwError = false;
+
+      @Override
+      public Pipeline createPipeline(ReplicationConfig repConf,
+          List<DatanodeDetails> excludedNodes,
+          List<DatanodeDetails> favoredNodes) throws IOException {
+        if (throwError) {
+          throw new IOException("Cannot create pipelines");
+        }
+        throwError = true;
+        return super.createPipeline(repConfig);
+      }
+    };
+    provider = new WritableECContainerProvider(
+        conf, pipelineManager, containerManager, pipelineChoosingPolicy);
+
+    ContainerInfo container =
+        provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+    for (int i = 0; i < 5; i++) {
+      ContainerInfo nextContainer =
+          provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+      assertEquals(container, nextContainer);
+    }
+  }
+
+  @Test
+  public void testNewContainerAllocatedAndPipelinesClosedIfNoSpaceInExisting()
+      throws IOException {
+    Set<ContainerInfo> allocatedContainers = new HashSet<>();
+    for (int i = 0; i < minPipelines; i++) {
+      ContainerInfo container =
+          provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+      assertFalse(allocatedContainers.contains(container));
+      allocatedContainers.add(container);
+    }
+    // Update all the containers to make them nearly full, but with enough space
+    // for an EC block to be striped across them.
+    for (ContainerInfo c : allocatedContainers) {
+      c.setUsedBytes(getMaxContainerSize() - 30 * 1024 * 1024);
+    }
+
+    // Get a new container of size 50 and ensure it is one of the original set.
+    // We ask for a space of 50, but as it is stripped across the EC group it
+    // will actually need 50 / dataNum space
+    ContainerInfo newContainer =
+        provider.getContainer(50 * 1024 * 1024, repConfig, OWNER,
+            new ExcludeList());
+    assertNotNull(newContainer);
+    assertTrue(allocatedContainers.contains(newContainer));
+    // Now get a new container where there is not enough space in the existing
+    // and ensure a new container gets created.
+    newContainer = provider.getContainer(
+        128 * 1024 * 1024, repConfig, OWNER, new ExcludeList());
+    assertNotNull(newContainer);
+    assertFalse(allocatedContainers.contains(newContainer));
+    // The original pipelines should all be closed, triggered by the lack of
+    // space.
+    for (ContainerInfo c : allocatedContainers) {
+      Pipeline pipeline = pipelineManager.getPipeline(c.getPipelineID());
+      assertEquals(CLOSED, pipeline.getPipelineState());
+    }
+  }
+
+  @Test
+  public void testPipelineNotFoundWhenAttemptingToUseExisting()
+      throws IOException {
+    // Ensure PM throws PNF exception when we ask for the containers in the
+    // pipeline
+    pipelineManager = new MockPipelineManager(
+        dbStore, scmhaManager, nodeManager) {
+
+      @Override
+      public NavigableSet<ContainerID> getContainersInPipeline(
+          PipelineID pipelineID) throws IOException {
+        throw new PipelineNotFoundException("Simulated exception");
+      }
+    };
+    provider = new WritableECContainerProvider(
+        conf, pipelineManager, containerManager, pipelineChoosingPolicy);
+
+    Set<ContainerInfo> allocatedContainers = new HashSet<>();
+    for (int i = 0; i < minPipelines; i++) {
+      ContainerInfo container =
+          provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+      assertFalse(allocatedContainers.contains(container));
+      allocatedContainers.add(container);
+    }
+    // Now attempt to get a container - any attempt to use an existing with
+    // throw PNF and then we must allocate a new one
+    ContainerInfo newContainer =
+        provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+    assertNotNull(newContainer);
+    assertFalse(allocatedContainers.contains(newContainer));
+  }
+
+  @Test
+  public void testContainerNotFoundWhenAttemptingToUseExisting()
+      throws IOException {
+    Set<ContainerInfo> allocatedContainers = new HashSet<>();
+    for (int i = 0; i < minPipelines; i++) {
+      ContainerInfo container =
+          provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+      assertFalse(allocatedContainers.contains(container));
+      allocatedContainers.add(container);
+    }
+
+    // Ensure ContainerManager always throws when a container is requested so
+    // existing pipelines cannot be used
+    Mockito.doAnswer(call -> {
+      throw new ContainerNotFoundException();
+    }).when(containerManager).getContainer(Matchers.any(ContainerID.class));
+
+    ContainerInfo newContainer =
+        provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+    assertNotNull(newContainer);
+    assertFalse(allocatedContainers.contains(newContainer));
+
+    // Ensure all the existing pipelines are closed
+    for (ContainerInfo c : allocatedContainers) {
+      Pipeline pipeline = pipelineManager.getPipeline(c.getPipelineID());
+      assertEquals(CLOSED, pipeline.getPipelineState());
+    }
+  }
+
+  @Test
+  public void testPipelineOpenButContainerRemovedFromIt() throws IOException {
+    // This can happen if the container close process is triggered from the DN.
+    // When tha happens, CM will change the container state to CLOSING and
+    // remove it from the container list in pipeline Manager.
+    Set<ContainerInfo> allocatedContainers = new HashSet<>();
+    for (int i = 0; i < minPipelines; i++) {
+      ContainerInfo container = provider.getContainer(
+          1, repConfig, OWNER, new ExcludeList());
+      assertFalse(allocatedContainers.contains(container));
+      allocatedContainers.add(container);
+      // Remove the container from the pipeline to simulate closing it
+      pipelineManager.removeContainerFromPipeline(
+          container.getPipelineID(), container.containerID());
+    }
+    ContainerInfo newContainer = provider.getContainer(
+        1, repConfig, OWNER, new ExcludeList());
+    assertFalse(allocatedContainers.contains(newContainer));
+    for (ContainerInfo c : allocatedContainers) {
+      Pipeline pipeline = pipelineManager.getPipeline(c.getPipelineID());
+      assertEquals(CLOSED, pipeline.getPipelineState());
+    }
+  }
+
+  @Test
+  public void testExcludedNodesPassedToCreatePipelineIfProvided()
+      throws IOException {
+    PipelineManager pipelineManagerSpy = Mockito.spy(pipelineManager);
+    provider = new WritableECContainerProvider(
+        conf, pipelineManagerSpy, containerManager, pipelineChoosingPolicy);
+    ExcludeList excludeList = new ExcludeList();
+
+    // EmptyList should be passed if there are no nodes excluded.
+    ContainerInfo container = provider.getContainer(
+        1, repConfig, OWNER, excludeList);
+    assertNotNull(container);
+
+    verify(pipelineManagerSpy).createPipeline(repConfig,
+        Collections.emptyList(), Collections.emptyList());
+
+    // If nodes are excluded then the excluded nodes should be passed through to
+    // the create pipeline call.
+    excludeList.addDatanode(MockDatanodeDetails.randomDatanodeDetails());
+    List<DatanodeDetails> excludedNodes =
+        new ArrayList<>(excludeList.getDatanodes());
+
+    container = provider.getContainer(
+        1, repConfig, OWNER, excludeList);
+    assertNotNull(container);
+    verify(pipelineManagerSpy).createPipeline(repConfig, excludedNodes,
+        Collections.emptyList());
+  }
+
+  private ContainerInfo createContainer(Pipeline pipeline,
+      ReplicationConfig repConf, long containerID) {
+    return new ContainerInfo.Builder()
+        .setContainerID(containerID)
+        .setOwner(OWNER)
+        .setReplicationConfig(repConf)
+        .setState(HddsProtos.LifeCycleState.OPEN)
+        .setPipelineID(pipeline.getId())
+        .setNumberOfKeys(0)
+        .setUsedBytes(0)
+        .setSequenceId(0)
+        .setDeleteTransactionId(0)
+        .build();
+  }
+
+  private long getMaxContainerSize() {
+    return (long)conf.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, BYTES);
+  }
+
+}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
index af7337a..d72682a 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
@@ -21,6 +21,7 @@
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto;
@@ -409,9 +410,10 @@
   @Override
   public List<ContainerInfo> listContainer(long startContainerID,
       int count, HddsProtos.LifeCycleState state,
-      HddsProtos.ReplicationFactor factor) throws IOException {
+      HddsProtos.ReplicationType repType,
+      ReplicationConfig replicationConfig) throws IOException {
     return storageContainerLocationClient.listContainer(
-        startContainerID, count, state, factor);
+        startContainerID, count, state, repType, replicationConfig);
   }
 
   /**
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
index fd97153d..37bea18 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
@@ -124,6 +124,9 @@
   private static String buildReplicaDetails(ContainerReplicaInfo replica) {
     StringBuilder sb = new StringBuilder();
     sb.append("State: " + replica.getState() + ";");
+    if (replica.getReplicaIndex() != -1) {
+      sb.append(" ReplicaIndex: " + replica.getReplicaIndex() + ";");
+    }
     sb.append(" Origin: " + replica.getPlaceOfBirth().toString() + ";");
     sb.append(" Location: "
         + buildDatanodeDetails(replica.getDatanodeDetails()));
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
index 3d776bb..b120fe4 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
@@ -20,7 +20,11 @@
 import java.io.IOException;
 import java.util.List;
 
+import com.google.common.base.Strings;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
@@ -65,9 +69,14 @@
           "DELETING, DELETED)")
   private HddsProtos.LifeCycleState state;
 
-  @Option(names = {"--factor"},
-      description = "Container factor(ONE, THREE)")
-  private HddsProtos.ReplicationFactor factor;
+  @Option(names = {"-t", "--type"},
+      description = "Replication Type (RATIS, STAND_ALONE or EC)")
+  private HddsProtos.ReplicationType type;
+
+  @Option(names = {"-r", "--replication", "--factor"},
+      description = "Container replication (ONE, THREE for Ratis, " +
+          "rs-6-3-1024k for EC)")
+  private String replication;
 
   private static final ObjectWriter WRITER;
 
@@ -90,8 +99,19 @@
 
   @Override
   public void execute(ScmClient scmClient) throws IOException {
+    if (!Strings.isNullOrEmpty(replication) && type == null) {
+      // Set type to RATIS as that is what any command prior to this change
+      // would have expected.
+      type = HddsProtos.ReplicationType.RATIS;
+    }
+    ReplicationConfig repConfig = null;
+    if (!Strings.isNullOrEmpty(replication)) {
+      repConfig = ReplicationConfig.parse(
+          ReplicationType.fromProto(type),
+          replication, new OzoneConfiguration());
+    }
     List<ContainerInfo> containerList =
-        scmClient.listContainer(startId, count, state, factor);
+        scmClient.listContainer(startId, count, state, type, repConfig);
 
     // Output data list
     for (ContainerInfo container : containerList) {
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java
index faea21e..86299ee 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java
@@ -55,7 +55,14 @@
 
   @Override
   public void execute(ScmClient scmClient) throws IOException {
-    if (type == HddsProtos.ReplicationType.CHAINED) {
+    // Once we support creating EC containers/pipelines from the client, the
+    // client should check if SCM is able to fulfil the request, and
+    // understands an EcReplicationConfig. For that we also need to have SCM's
+    // version here from ScmInfo response.
+    // As I see there is no way to specify ECReplicationConfig properly here
+    // so failing the request if type is EC, seems to be safe.
+    if (type == HddsProtos.ReplicationType.CHAINED
+        || type == HddsProtos.ReplicationType.EC) {
       throw new IllegalArgumentException(type.name()
           + " is not supported yet.");
     }
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java
index a91af50..28e0eb9 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java
@@ -21,6 +21,8 @@
 import com.google.common.base.Strings;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -39,23 +41,42 @@
     versionProvider = HddsVersionProvider.class)
 public class ListPipelinesSubcommand extends ScmSubcommand {
 
-  @CommandLine.Option(names = {"-ffc", "--filterByFactor"},
-      description = "Filter listed pipelines by Factor(ONE/one)",
+  @CommandLine.Option(names = {"-t", "--type"},
+      description = "Filter listed pipelines by replication type, RATIS or EC",
       defaultValue = "")
-  private String factor;
+  private String replicationType;
 
-  @CommandLine.Option(names = {"-fst", "--filterByState"},
-      description = "Filter listed pipelines by State(OPEN/CLOSE)",
+  @CommandLine.Option(
+      names = {"-r", "--replication", "-ffc", "--filterByFactor"},
+      description = "Filter listed pipelines by replication, eg ONE, THREE or "
+      + "for EC rs-3-2-1024k",
+      defaultValue = "")
+  private String replication;
+
+  @CommandLine.Option(names = {"-s", "--state", "-fst", "--filterByState"},
+      description = "Filter listed pipelines by State, eg OPEN, CLOSED",
       defaultValue = "")
   private String state;
 
   @Override
   public void execute(ScmClient scmClient) throws IOException {
     Stream<Pipeline> stream = scmClient.listPipelines().stream();
-    if (!Strings.isNullOrEmpty(factor)) {
+    if (!Strings.isNullOrEmpty(replication)) {
+      if (Strings.isNullOrEmpty(replicationType)) {
+        throw new IOException(
+            "ReplicationType cannot be null if replication is passed");
+      }
+      ReplicationConfig repConfig =
+          ReplicationConfig.parse(ReplicationType.valueOf(replicationType),
+              replication, new OzoneConfiguration());
       stream = stream.filter(
-          p -> ReplicationConfig.getLegacyFactor(p.getReplicationConfig())
-              .toString().compareToIgnoreCase(factor) == 0);
+          p -> p.getReplicationConfig().equals(repConfig));
+    } else if (!Strings.isNullOrEmpty(replicationType)) {
+      stream = stream.filter(
+          p -> p.getReplicationConfig()
+              .getReplicationType()
+              .toString()
+              .compareToIgnoreCase(replicationType) == 0);
     }
     if (!Strings.isNullOrEmpty(state)) {
       stream = stream.filter(p -> p.getPipelineState().toString()
diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestInfoSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestInfoSubCommand.java
index 23a1a0b..c604019 100644
--- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestInfoSubCommand.java
+++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestInfoSubCommand.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdds.scm.cli.container;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -81,8 +82,19 @@
 
   @Test
   public void testReplicasIncludedInOutput() throws Exception {
+    testReplicaIncludedInOutput(false);
+  }
+
+  @Test
+  public void testReplicaIndexInOutput() throws Exception {
+    testReplicaIncludedInOutput(true);
+  }
+
+
+  private void testReplicaIncludedInOutput(boolean includeIndex)
+      throws IOException {
     Mockito.when(scmClient.getContainerReplicas(anyLong()))
-        .thenReturn(getReplicas());
+        .thenReturn(getReplicas(includeIndex));
     cmd = new InfoSubcommand();
     CommandLine c = new CommandLine(cmd);
     c.parseArgs("1", "--replicas");
@@ -94,7 +106,7 @@
         .filter(m -> m.getRenderedMessage().matches("(?s)^Replicas:.*"))
         .collect(Collectors.toList());
     Assert.assertEquals(1, replica.size());
-    
+
     // Ensure each DN UUID is mentioned in the message:
     for (DatanodeDetails dn : datanodes) {
       Pattern pattern = Pattern.compile(".*" + dn.getUuid().toString() + ".*",
@@ -102,6 +114,11 @@
       Matcher matcher = pattern.matcher(replica.get(0).getRenderedMessage());
       Assert.assertTrue(matcher.matches());
     }
+    // Ensure ReplicaIndex is not mentioned as it was not passed in the proto:
+    Pattern pattern = Pattern.compile(".*ReplicaIndex.*",
+        Pattern.DOTALL);
+    Matcher matcher = pattern.matcher(replica.get(0).getRenderedMessage());
+    Assert.assertEquals(includeIndex, matcher.matches());
   }
 
   @Test
@@ -151,7 +168,20 @@
   @Test
   public void testReplicasOutputWithJson() throws IOException {
     Mockito.when(scmClient.getContainerReplicas(anyLong()))
-        .thenReturn(getReplicas());
+        .thenReturn(getReplicas(true));
+    testJsonOutput();
+  }
+
+  @Test
+  public void testECContainerReplicasOutputWithJson() throws IOException {
+    Mockito.when(scmClient.getContainerReplicas(anyLong()))
+        .thenReturn(getReplicas(true));
+    Mockito.when(scmClient.getContainerWithPipeline(anyLong()))
+        .thenReturn(getECContainerWithPipeline());
+    testJsonOutput();
+  }
+
+  private void testJsonOutput() throws IOException {
     cmd = new InfoSubcommand();
     CommandLine c = new CommandLine(cmd);
     c.parseArgs("1", "--json");
@@ -169,20 +199,29 @@
       Matcher matcher = pattern.matcher(json);
       Assert.assertTrue(matcher.matches());
     }
+    Pattern pattern = Pattern.compile(".*replicaIndex.*",
+        Pattern.DOTALL);
+    Matcher matcher = pattern.matcher(json);
+    Assert.assertTrue(matcher.matches());
   }
 
-  private List<ContainerReplicaInfo> getReplicas() {
+
+  private List<ContainerReplicaInfo> getReplicas(boolean includeIndex) {
     List<ContainerReplicaInfo> replicas = new ArrayList<>();
     for (DatanodeDetails dn : datanodes) {
-      ContainerReplicaInfo container =  new ContainerReplicaInfo.Builder()
+      ContainerReplicaInfo.Builder container
+          = new ContainerReplicaInfo.Builder()
           .setContainerID(1)
           .setBytesUsed(1234)
           .setState("CLOSED")
           .setPlaceOfBirth(dn.getUuid())
           .setDatanodeDetails(dn)
           .setKeyCount(1)
-          .setSequenceId(1).build();
-      replicas.add(container);
+          .setSequenceId(1);
+      if (includeIndex) {
+        container.setReplicaIndex(4);
+      }
+      replicas.add(container.build());
     }
     return replicas;
   }
@@ -207,6 +246,26 @@
     return new ContainerWithPipeline(container, pipeline);
   }
 
+  private ContainerWithPipeline getECContainerWithPipeline() {
+    Pipeline pipeline = new Pipeline.Builder()
+        .setState(Pipeline.PipelineState.CLOSED)
+        .setReplicationConfig(new ECReplicationConfig(3, 2))
+        .setId(PipelineID.randomId())
+        .setNodes(datanodes)
+        .build();
+
+    ContainerInfo container = new ContainerInfo.Builder()
+        .setSequenceId(1)
+        .setPipelineID(pipeline.getId())
+        .setUsedBytes(1234)
+        .setReplicationConfig(RatisReplicationConfig.getInstance(THREE))
+        .setNumberOfKeys(1)
+        .setState(CLOSED)
+        .build();
+
+    return new ContainerWithPipeline(container, pipeline);
+  }
+
   private List<DatanodeDetails> createDatanodeDetails(int count) {
     List<DatanodeDetails> dns = new ArrayList<>();
     for (int i = 0; i < count; i++) {
diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/pipeline/TestListPipelinesSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/pipeline/TestListPipelinesSubCommand.java
new file mode 100644
index 0000000..acc3bda
--- /dev/null
+++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/pipeline/TestListPipelinesSubCommand.java
@@ -0,0 +1,192 @@
+/*
+ * 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.hdds.scm.cli.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import picocli.CommandLine;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the ListPipelineSubCommand class.
+ */
+public class TestListPipelinesSubCommand {
+
+  private ListPipelinesSubcommand cmd;
+  private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private final ByteArrayOutputStream errContent = new ByteArrayOutputStream();
+  private final PrintStream originalOut = System.out;
+  private final PrintStream originalErr = System.err;
+  private static final String DEFAULT_ENCODING = StandardCharsets.UTF_8.name();
+  private ScmClient scmClient;
+
+  @Before
+  public void setup() throws IOException {
+    cmd = new ListPipelinesSubcommand();
+    System.setOut(new PrintStream(outContent, false, DEFAULT_ENCODING));
+    System.setErr(new PrintStream(errContent, false, DEFAULT_ENCODING));
+
+    scmClient = mock(ScmClient.class);
+    Mockito.when(scmClient.listPipelines())
+        .thenAnswer(invocation -> createPipelines());
+  }
+
+  @After
+  public void tearDown() {
+    System.setOut(originalOut);
+    System.setErr(originalErr);
+  }
+
+  @Test
+  public void testAllPipelinesReturnedWithNoFilter() throws IOException {
+    CommandLine c = new CommandLine(cmd);
+    c.parseArgs();
+    cmd.execute(scmClient);
+    Assert.assertEquals(6, outContent.toString(DEFAULT_ENCODING)
+        .split(System.getProperty("line.separator")).length);
+  }
+
+  @Test
+  public void testOnlyOpenReturned() throws IOException {
+    CommandLine c = new CommandLine(cmd);
+    c.parseArgs("-s", "OPEN");
+    cmd.execute(scmClient);
+    String output = outContent.toString(DEFAULT_ENCODING);
+    Assert.assertEquals(3, output.split(
+        System.getProperty("line.separator")).length);
+    Assert.assertEquals(-1, output.indexOf("CLOSED"));
+  }
+
+  @Test(expected = IOException.class)
+  public void testExceptionIfReplicationWithoutType() throws IOException {
+    CommandLine c = new CommandLine(cmd);
+    c.parseArgs("-r", "THREE");
+    cmd.execute(scmClient);
+  }
+
+  @Test
+  public void testReplicationAndType() throws IOException {
+    CommandLine c = new CommandLine(cmd);
+    c.parseArgs("-r", "THREE", "-t", "RATIS");
+    cmd.execute(scmClient);
+
+    String output = outContent.toString(DEFAULT_ENCODING);
+    Assert.assertEquals(2, output.split(
+        System.getProperty("line.separator")).length);
+    Assert.assertEquals(-1, output.indexOf("EC"));
+  }
+
+  @Test
+  public void testReplicationAndTypeEC() throws IOException {
+    CommandLine c = new CommandLine(cmd);
+    c.parseArgs("-r", "rs-6-3-1024k", "-t", "EC");
+    cmd.execute(scmClient);
+
+    String output = outContent.toString(DEFAULT_ENCODING);
+    Assert.assertEquals(1, output.split(
+        System.getProperty("line.separator")).length);
+    Assert.assertEquals(-1,
+        output.indexOf("ReplicationConfig: RATIS"));
+  }
+
+  @Test
+  public void testReplicationAndTypeAndState() throws IOException {
+    CommandLine c = new CommandLine(cmd);
+    c.parseArgs("-r", "THREE", "-t", "RATIS", "-s", "OPEN");
+    cmd.execute(scmClient);
+
+    String output = outContent.toString(DEFAULT_ENCODING);
+    Assert.assertEquals(1, output.split(
+        System.getProperty("line.separator")).length);
+    Assert.assertEquals(-1, output.indexOf("CLOSED"));
+    Assert.assertEquals(-1, output.indexOf("EC"));
+  }
+
+  private List<Pipeline> createPipelines() {
+    List<Pipeline> pipelines = new ArrayList<>();
+    pipelines.add(createPipeline(StandaloneReplicationConfig.getInstance(ONE),
+        Pipeline.PipelineState.OPEN));
+    pipelines.add(createPipeline(RatisReplicationConfig.getInstance(THREE),
+        Pipeline.PipelineState.OPEN));
+    pipelines.add(createPipeline(RatisReplicationConfig.getInstance(THREE),
+        Pipeline.PipelineState.CLOSED));
+
+    pipelines.add(createPipeline(
+        new ECReplicationConfig(3, 2), Pipeline.PipelineState.OPEN));
+    pipelines.add(createPipeline(
+        new ECReplicationConfig(3, 2), Pipeline.PipelineState.CLOSED));
+    pipelines.add(createPipeline(
+        new ECReplicationConfig(6, 3), Pipeline.PipelineState.CLOSED));
+
+    return pipelines;
+  }
+
+  private Pipeline createPipeline(ReplicationConfig repConfig,
+      Pipeline.PipelineState state) {
+    return new Pipeline.Builder()
+        .setId(PipelineID.randomId())
+        .setCreateTimestamp(System.currentTimeMillis())
+        .setState(state)
+        .setReplicationConfig(repConfig)
+        .setNodes(createDatanodeDetails(1))
+        .build();
+  }
+
+  private List<DatanodeDetails> createDatanodeDetails(int count) {
+    List<DatanodeDetails> dns = new ArrayList<>();
+    for (int i = 0; i < count; i++) {
+      HddsProtos.DatanodeDetailsProto dnd =
+          HddsProtos.DatanodeDetailsProto.newBuilder()
+              .setHostName("host" + i)
+              .setIpAddress("1.2.3." + i + 1)
+              .setNetworkLocation("/default")
+              .setNetworkName("host" + i)
+              .addPorts(HddsProtos.Port.newBuilder()
+                  .setName("ratis").setValue(5678).build())
+              .setUuid(UUID.randomUUID().toString())
+              .build();
+      dns.add(DatanodeDetails.getFromProtoBuf(dnd));
+    }
+    return dns;
+  }
+}
+
+
diff --git a/hadoop-ozone/client/pom.xml b/hadoop-ozone/client/pom.xml
index 9ba3af6..f072308 100644
--- a/hadoop-ozone/client/pom.xml
+++ b/hadoop-ozone/client/pom.xml
@@ -39,6 +39,10 @@
       <artifactId>ozone-common</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-erasurecode</artifactId>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
index 43d28f0..6876059 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.client;
 
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -56,6 +57,7 @@
    * Bucket encryption key name.
    */
   private String bucketEncryptionKey;
+  private DefaultReplicationConfig defaultReplicationConfig;
   private final String sourceVolume;
   private final String sourceBucket;
 
@@ -80,15 +82,16 @@
    * @param sourceBucket
    * @param quotaInBytes Bucket quota in bytes.
    * @param quotaInNamespace Bucket quota in counts.
-   * @param bucketLayout Bucket Layouts.
+   * @param bucketLayout bucket layout.
    * @param owner owner of the bucket.
+   * @param defaultReplicationConfig default replication config.
    */
   @SuppressWarnings("parameternumber")
   private BucketArgs(Boolean versioning, StorageType storageType,
       List<OzoneAcl> acls, Map<String, String> metadata,
       String bucketEncryptionKey, String sourceVolume, String sourceBucket,
       long quotaInBytes, long quotaInNamespace, BucketLayout bucketLayout,
-      String owner) {
+      String owner, DefaultReplicationConfig defaultReplicationConfig) {
     this.acls = acls;
     this.versioning = versioning;
     this.storageType = storageType;
@@ -100,6 +103,7 @@
     this.quotaInNamespace = quotaInNamespace;
     this.bucketLayout = bucketLayout;
     this.owner = owner;
+    this.defaultReplicationConfig = defaultReplicationConfig;
   }
 
   /**
@@ -144,6 +148,14 @@
   }
 
   /**
+   * Returns the bucket default replication config.
+   * @return bucket's default Replication Config.
+   */
+  public DefaultReplicationConfig getDefaultReplicationConfig() {
+    return this.defaultReplicationConfig;
+  }
+
+  /**
    * Returns new builder class that builds a OmBucketInfo.
    *
    * @return Builder
@@ -205,6 +217,7 @@
     private long quotaInNamespace;
     private BucketLayout bucketLayout;
     private String owner;
+    private DefaultReplicationConfig defaultReplicationConfig;
 
     public Builder() {
       metadata = new HashMap<>();
@@ -267,6 +280,12 @@
       return this;
     }
 
+    public BucketArgs.Builder setDefaultReplicationConfig(
+        DefaultReplicationConfig defaultReplConfig) {
+      defaultReplicationConfig = defaultReplConfig;
+      return this;
+    }
+
     /**
      * Constructs the BucketArgs.
      * @return instance of BucketArgs.
@@ -274,7 +293,7 @@
     public BucketArgs build() {
       return new BucketArgs(versioning, storageType, acls, metadata,
           bucketEncryptionKey, sourceVolume, sourceBucket, quotaInBytes,
-          quotaInNamespace, bucketLayout, owner);
+          quotaInNamespace, bucketLayout, owner, defaultReplicationConfig);
     }
   }
 }
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index e49aeb7..9ae4185 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -22,6 +22,7 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -80,7 +81,7 @@
   /**
    * Default replication factor to be used while creating keys.
    */
-  private final ReplicationConfig defaultReplication;
+  private ReplicationConfig defaultReplication;
 
   /**
    * Type of storage to be used for this bucket.
@@ -151,7 +152,8 @@
     this.volumeName = volumeName;
     this.name = bucketName;
 
-    this.defaultReplication = ReplicationConfig.getDefault(conf);
+    // Bucket level replication is not configured by default.
+    this.defaultReplication = null;
 
     this.proxy = proxy;
     this.ozoneObj = OzoneObjInfo.Builder.newBuilder()
@@ -225,6 +227,32 @@
     this.bucketLayout = bucketLayout;
   }
 
+  @SuppressWarnings("parameternumber")
+  public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
+      String volumeName, String bucketName, StorageType storageType,
+      Boolean versioning, long creationTime, long modificationTime,
+      Map<String, String> metadata, String encryptionKeyName,
+      String sourceVolume, String sourceBucket, long usedBytes,
+      long usedNamespace, long quotaInBytes, long quotaInNamespace,
+      BucketLayout bucketLayout, String owner,
+      DefaultReplicationConfig defaultReplicationConfig) {
+    this(conf, proxy, volumeName, bucketName, storageType, versioning,
+        creationTime, modificationTime, metadata, encryptionKeyName,
+        sourceVolume, sourceBucket, usedBytes, usedNamespace, quotaInBytes,
+        quotaInNamespace, bucketLayout, owner);
+    this.bucketLayout = bucketLayout;
+    if (defaultReplicationConfig != null) {
+      this.defaultReplication =
+          defaultReplicationConfig.getType() == ReplicationType.EC ?
+              defaultReplicationConfig.getEcReplicationConfig() :
+              ReplicationConfig
+                  .fromTypeAndFactor(defaultReplicationConfig.getType(),
+                      defaultReplicationConfig.getFactor());
+    } else {
+      // Bucket level replication is not configured by default.
+      this.defaultReplication = null;
+    }
+  }
 
   @SuppressWarnings("checkstyle:ParameterNumber")
   public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
@@ -508,6 +536,17 @@
   }
 
   /**
+   * Sets/Changes the default replication config of this Bucket.
+   *
+   * @param replicationConfig Replication config that can be applied to bucket.
+   * @throws IOException
+   */
+  public void setReplicationConfig(ReplicationConfig replicationConfig)
+      throws IOException {
+    proxy.setReplicationConfig(volumeName, name, replicationConfig);
+  }
+
+  /**
    * Creates a new key in the bucket, with default replication type RATIS and
    * with replication factor THREE.
    * @param key Name of the key to be created.
@@ -1269,4 +1308,8 @@
   public BucketLayout getBucketLayout() {
     return bucketLayout;
   }
+
+  public ReplicationConfig getReplicationConfig() {
+    return this.defaultReplication;
+  }
 }
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java
index e37969d..f0302c8 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.client;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
@@ -154,12 +155,14 @@
    */
 
   @Deprecated
+  @JsonIgnore
   public ReplicationType getReplicationType() {
     return ReplicationType
             .fromProto(replicationConfig.getReplicationType());
   }
 
   @Deprecated
+  @JsonIgnore
   public int getReplicationFactor() {
     return replicationConfig.getRequiredNodes();
   }
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.java
new file mode 100644
index 0000000..dc8f7b0
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.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.ozone.client.io;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+
+import java.io.IOException;
+
+/**
+ * Exception used to indicate a problem with a specific block location, allowing
+ * the failed location to be communicated back to the caller.
+ */
+public class BadDataLocationException extends IOException {
+
+  private DatanodeDetails failedLocation;
+
+  public BadDataLocationException(DatanodeDetails dn) {
+    super();
+    failedLocation = dn;
+  }
+
+  public BadDataLocationException(DatanodeDetails dn, String message) {
+    super(message);
+    failedLocation = dn;
+  }
+
+  public BadDataLocationException(DatanodeDetails dn, String message,
+      Throwable ex) {
+    super(message, ex);
+    failedLocation = dn;
+  }
+
+  public BadDataLocationException(DatanodeDetails dn, Throwable ex) {
+    super(ex);
+    failedLocation = dn;
+  }
+
+  public DatanodeDetails getFailedLocation() {
+    return failedLocation;
+  }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
new file mode 100644
index 0000000..d1bf7f3
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
@@ -0,0 +1,55 @@
+/*
+ * 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.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+
+import java.util.function.Function;
+
+/**
+ * Interface used by classes which need to obtain BlockStream instances.
+ */
+public interface BlockInputStreamFactory {
+
+  /**
+   * Create a new BlockInputStream based on the replication Config. If the
+   * replication Config indicates the block is EC, then it will create an
+   * ECBlockInputStream, otherwise a BlockInputStream will be returned.
+   * @param repConfig The replication Config
+   * @param blockInfo The blockInfo representing the block.
+   * @param pipeline The pipeline to be used for reading the block
+   * @param token The block Access Token
+   * @param verifyChecksum Whether to verify checksums or not.
+   * @param xceiverFactory Factory to create the xceiver in the client
+   * @param refreshFunction Function to refresh the pipeline if needed
+   * @return BlockExtendedInputStream of the correct type.
+   */
+  BlockExtendedInputStream create(ReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, Pipeline pipeline,
+      Token<OzoneBlockTokenIdentifier> token, boolean verifyChecksum,
+       XceiverClientFactory xceiverFactory,
+       Function<BlockID, Pipeline> refreshFunction);
+
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
new file mode 100644
index 0000000..104e5bc
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
@@ -0,0 +1,92 @@
+/*
+ * 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.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Factory class to create various BlockStream instances.
+ */
+public class BlockInputStreamFactoryImpl implements BlockInputStreamFactory {
+
+  private ECBlockInputStreamFactory ecBlockStreamFactory;
+
+  public static BlockInputStreamFactory getInstance(
+      ByteBufferPool byteBufferPool,
+      Supplier<ExecutorService> ecReconstructExecutorSupplier) {
+    return new BlockInputStreamFactoryImpl(byteBufferPool,
+        ecReconstructExecutorSupplier);
+  }
+
+  public BlockInputStreamFactoryImpl() {
+    this(new ElasticByteBufferPool(), Executors::newSingleThreadExecutor);
+  }
+
+  public BlockInputStreamFactoryImpl(ByteBufferPool byteBufferPool,
+      Supplier<ExecutorService> ecReconstructExecutorSupplier) {
+    this.ecBlockStreamFactory =
+        ECBlockInputStreamFactoryImpl.getInstance(this, byteBufferPool,
+            ecReconstructExecutorSupplier);
+  }
+
+  /**
+   * Create a new BlockInputStream based on the replication Config. If the
+   * replication Config indicates the block is EC, then it will create an
+   * ECBlockInputStream, otherwise a BlockInputStream will be returned.
+   * @param repConfig The replication Config
+   * @param blockInfo The blockInfo representing the block.
+   * @param pipeline The pipeline to be used for reading the block
+   * @param token The block Access Token
+   * @param verifyChecksum Whether to verify checksums or not.
+   * @param xceiverFactory Factory to create the xceiver in the client
+   * @param refreshFunction Function to refresh the pipeline if needed
+   * @return BlockExtendedInputStream of the correct type.
+   */
+  public BlockExtendedInputStream create(ReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, Pipeline pipeline,
+      Token<OzoneBlockTokenIdentifier> token, boolean verifyChecksum,
+      XceiverClientFactory xceiverFactory,
+      Function<BlockID, Pipeline> refreshFunction) {
+    if (repConfig.getReplicationType().equals(HddsProtos.ReplicationType.EC)) {
+      return new ECBlockInputStreamProxy((ECReplicationConfig)repConfig,
+          blockInfo, verifyChecksum, xceiverFactory, refreshFunction,
+          ecBlockStreamFactory);
+    } else {
+      return new BlockInputStream(blockInfo.getBlockID(), blockInfo.getLength(),
+          pipeline, token, verifyChecksum, xceiverFactory, refreshFunction);
+    }
+  }
+
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java
index 5aff685..2b3b756 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java
@@ -42,7 +42,7 @@
  *
  * The base implementation is handling Ratis-3 writes, with a single stream,
  * but there can be other implementations that are using a different way.
- * */
+ */
 public class BlockOutputStreamEntry extends OutputStream {
 
   private final OzoneClientConfig config;
@@ -276,7 +276,8 @@
    * here.
    * @param id the last know ID of the block.
    */
-  void updateBlockID(BlockID id) {
+  @VisibleForTesting
+  protected void updateBlockID(BlockID id) {
     this.blockID = id;
   }
 
@@ -352,13 +353,11 @@
       return this;
     }
 
-
     public Builder setLength(long len) {
       this.length = len;
       return this;
     }
 
-
     public Builder setBufferPool(BufferPool pool) {
       this.bufferPool = pool;
       return this;
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
index 7985bf7..bdfa3f2 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
@@ -102,7 +102,7 @@
         .setMultipartUploadPartNumber(partNumber).build();
     this.requestID = requestId;
     this.openID = openID;
-    this.excludeList = new ExcludeList();
+    this.excludeList = createExcludeList();
 
     this.bufferPool =
         new BufferPool(config.getStreamBufferSize(),
@@ -112,6 +112,10 @@
                 .createByteBufferConversion(unsafeByteBufferConversion));
   }
 
+  ExcludeList createExcludeList() {
+    return new ExcludeList();
+  }
+
   /**
    * A constructor for testing purpose only.
    *
@@ -216,7 +220,7 @@
                 .setLength(streamEntry.getCurrentPosition())
                 .setOffset(0)
                 .setToken(streamEntry.getToken())
-                .setPipeline(streamEntry.getPipelineForOMLocationReport())
+                .setPipeline(streamEntry.getPipeline())
                 .build();
         locationInfoList.add(info);
       }
@@ -319,7 +323,8 @@
     if (keyArgs != null) {
       // in test, this could be null
       long length = getKeyLength();
-      Preconditions.checkArgument(offset == length);
+      Preconditions.checkArgument(offset == length,
+          "Epected offset: " + offset + " expected len: " + length);
       keyArgs.setDataSize(length);
       keyArgs.setLocationInfoList(getLocationInfoList());
       // When the key is multipart upload part file upload, we should not
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
new file mode 100644
index 0000000..b0e9755
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
@@ -0,0 +1,420 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+/**
+ * Class to read data from an EC Block Group.
+ */
+public class ECBlockInputStream extends BlockExtendedInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockInputStream.class);
+
+  private final ECReplicationConfig repConfig;
+  private final int ecChunkSize;
+  private final long stripeSize;
+  private final BlockInputStreamFactory streamFactory;
+  private final boolean verifyChecksum;
+  private final XceiverClientFactory xceiverClientFactory;
+  private final Function<BlockID, Pipeline> refreshFunction;
+  private final OmKeyLocationInfo blockInfo;
+  private final DatanodeDetails[] dataLocations;
+  private final BlockExtendedInputStream[] blockStreams;
+  private final int maxLocations;
+
+  private long position = 0;
+  private boolean closed = false;
+  private boolean seeked = false;
+
+  protected OmKeyLocationInfo getBlockInfo() {
+    return blockInfo;
+  }
+
+  protected ECReplicationConfig getRepConfig() {
+    return repConfig;
+  }
+
+  protected DatanodeDetails[] getDataLocations() {
+    return dataLocations;
+  }
+
+  protected long getStripeSize() {
+    return stripeSize;
+  }
+
+  /**
+   * Returns the number of available data locations, taking account of the
+   * expected number of locations. Eg, if the block is less than 1 EC chunk,
+   * we only expect 1 data location. If it is between 1 and 2 chunks, we expect
+   * there to be 2 locations, and so on.
+   * @param expectedLocations The maximum number of allowed data locations,
+   *                          depending on the block size.
+   * @return The number of available data locations.
+   */
+  protected int availableDataLocations(int expectedLocations) {
+    int count = 0;
+    for (int i = 0; i < repConfig.getData() && i < expectedLocations; i++) {
+      if (dataLocations[i] != null) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  protected int availableParityLocations() {
+    int count = 0;
+    for (int i = repConfig.getData();
+         i < repConfig.getData() + repConfig.getParity(); i++) {
+      if (dataLocations[i] != null) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  public ECBlockInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    this.repConfig = repConfig;
+    this.ecChunkSize = repConfig.getEcChunkSize();
+    this.verifyChecksum = verifyChecksum;
+    this.blockInfo = blockInfo;
+    this.streamFactory = streamFactory;
+    this.xceiverClientFactory = xceiverClientFactory;
+    this.refreshFunction = refreshFunction;
+    this.maxLocations = repConfig.getData() + repConfig.getParity();
+    this.dataLocations = new DatanodeDetails[repConfig.getRequiredNodes()];
+    this.blockStreams =
+        new BlockExtendedInputStream[repConfig.getRequiredNodes()];
+
+    this.stripeSize = (long)ecChunkSize * repConfig.getData();
+    setBlockLocations(this.blockInfo.getPipeline());
+  }
+
+  public synchronized boolean hasSufficientLocations() {
+    // The number of locations needed is a function of the EC Chunk size. If the
+    // block length is <= the chunk size, we should only have location 1. If it
+    // is greater than the chunk size but less than chunk_size * 2, then we must
+    // have two locations. If it is greater than chunk_size * data_num, then we
+    // must have all data_num locations.
+    // We only consider data locations here.
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    return expectedDataBlocks == availableDataLocations(expectedDataBlocks);
+  }
+
+  protected int calculateExpectedDataBlocks(ECReplicationConfig rConfig) {
+    return ECBlockInputStreamProxy.expectedDataLocations(rConfig, getLength());
+  }
+
+  /**
+   * Using the current position, returns the index of the blockStream we should
+   * be reading from. This is the index in the internal array holding the
+   * stream reference. The block group index will be one greater than this.
+   * @return
+   */
+  protected int currentStreamIndex() {
+    return (int)((position / ecChunkSize) % repConfig.getData());
+  }
+
+  /**
+   * Uses the current position and ecChunkSize to determine which of the
+   * internal block streams the next read should come from. Also opens the
+   * stream if it has not been opened already.
+   * @return BlockInput stream to read from.
+   */
+  protected BlockExtendedInputStream getOrOpenStream(int locationIndex) {
+    BlockExtendedInputStream stream = blockStreams[locationIndex];
+    if (stream == null) {
+      // To read an EC block, we create a STANDALONE pipeline that contains the
+      // single location for the block index we want to read. The EC blocks are
+      // indexed from 1 to N, however the data locations are stored in the
+      // dataLocations array indexed from zero.
+      Pipeline pipeline = Pipeline.newBuilder()
+          .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+              HddsProtos.ReplicationFactor.ONE))
+          .setNodes(Arrays.asList(dataLocations[locationIndex]))
+          .setId(PipelineID.randomId())
+          .setState(Pipeline.PipelineState.CLOSED)
+          .build();
+
+      OmKeyLocationInfo blkInfo = new OmKeyLocationInfo.Builder()
+          .setBlockID(blockInfo.getBlockID())
+          .setLength(internalBlockLength(locationIndex + 1))
+          .setPipeline(blockInfo.getPipeline())
+          .setToken(blockInfo.getToken())
+          .setPartNumber(blockInfo.getPartNumber())
+          .build();
+      stream = streamFactory.create(
+          StandaloneReplicationConfig.getInstance(
+              HddsProtos.ReplicationFactor.ONE),
+          blkInfo, pipeline,
+          blockInfo.getToken(), verifyChecksum, xceiverClientFactory,
+          refreshFunction);
+      blockStreams[locationIndex] = stream;
+    }
+    return stream;
+  }
+
+  /**
+   * Returns the length of the Nth block in the block group, taking account of a
+   * potentially partial last stripe. Note that the internal block index is
+   * numbered starting from 1.
+   * @param index - Index number of the internal block, starting from 1
+   * @return
+   */
+  protected long internalBlockLength(int index) {
+    long lastStripe = blockInfo.getLength() % stripeSize;
+    long blockSize = (blockInfo.getLength() - lastStripe) / repConfig.getData();
+    long lastCell = lastStripe / ecChunkSize + 1;
+    long lastCellLength = lastStripe % ecChunkSize;
+
+    if (index > repConfig.getData()) {
+      // Its a parity block and their size is driven by the size of the
+      // first block of the block group. All parity blocks have the same size
+      // as block_1.
+      index = 1;
+    }
+
+    if (index < lastCell) {
+      return blockSize + ecChunkSize;
+    } else if (index == lastCell) {
+      return blockSize + lastCellLength;
+    } else {
+      return blockSize;
+    }
+  }
+
+  private void setBlockLocations(Pipeline pipeline) {
+    for (DatanodeDetails node : pipeline.getNodes()) {
+      int index = pipeline.getReplicaIndex(node);
+      addBlockLocation(index, node);
+    }
+  }
+
+  private void addBlockLocation(int index, DatanodeDetails location) {
+    if (index > maxLocations) {
+      throw new IndexOutOfBoundsException("The index " + index + " is greater "
+          + "than the EC Replication Config (" + repConfig + ")");
+    }
+    dataLocations[index - 1] = location;
+  }
+
+  protected long blockLength() {
+    return blockInfo.getLength();
+  }
+
+  protected long remaining() {
+    return blockLength() - position;
+  }
+
+  /**
+   * Read from the internal BlockInputStreams one EC cell at a time into the
+   * strategy buffer. This call may read from several internal BlockInputStreams
+   * if there is sufficient space in the buffer.
+   * @param strategy
+   * @return
+   * @throws IOException
+   */
+  @Override
+  protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+      throws IOException {
+    Preconditions.checkArgument(strategy != null);
+    checkOpen();
+
+    if (remaining() == 0) {
+      return EOF;
+    }
+
+    int totalRead = 0;
+    while (strategy.getTargetLength() > 0 && remaining() > 0) {
+      try {
+        int currentIndex = currentStreamIndex();
+        BlockExtendedInputStream stream = getOrOpenStream(currentIndex);
+        int read = readFromStream(stream, strategy);
+        totalRead += read;
+        position += read;
+      } catch (IOException ioe) {
+        throw new BadDataLocationException(
+            dataLocations[currentStreamIndex()], ioe);
+      }
+    }
+    return totalRead;
+  }
+
+  @Override
+  public synchronized long getRemaining() {
+    return blockInfo.getLength() - position;
+  }
+
+  @Override
+  public synchronized long getLength() {
+    return blockInfo.getLength();
+  }
+
+  @Override
+  public BlockID getBlockID() {
+    return blockInfo.getBlockID();
+  }
+
+  protected void seekStreamIfNecessary(BlockExtendedInputStream stream,
+      long partialChunkSize) throws IOException {
+    if (seeked) {
+      // Seek on the underlying streams is performed lazily, as there is a
+      // possibility a read after a seek may only read a small amount of data.
+      // Once this block stream has been seeked, we always check the position,
+      // but in the usual case, where there are no seeks at all, we don't need
+      // to do this extra work.
+      long basePosition = (position / stripeSize) * (long)ecChunkSize;
+      long streamPosition = basePosition + partialChunkSize;
+      if (streamPosition != stream.getPos()) {
+        // This ECBlockInputStream has been seeked, so the underlying
+        // block stream is no longer at the correct position. Therefore we need
+        // to seek it too.
+        stream.seek(streamPosition);
+      }
+    }
+  }
+
+  /**
+   * Read the most allowable amount of data from the current stream. This
+   * ensures we don't read past the end of an EC cell or the overall block
+   * group length.
+   * @param stream Stream to read from
+   * @param strategy The ReaderStrategy to read data into
+   * @return
+   * @throws IOException
+   */
+  private int readFromStream(BlockExtendedInputStream stream,
+      ByteReaderStrategy strategy)
+      throws IOException {
+    long partialPosition = position % ecChunkSize;
+    seekStreamIfNecessary(stream, partialPosition);
+    long ecLimit = ecChunkSize - partialPosition;
+    // Free space in the buffer to read into
+    long bufLimit = strategy.getTargetLength();
+    // How much we can read, the lower of the EC Cell, buffer and overall block
+    // remaining.
+    int expectedRead = (int)Math.min(Math.min(ecLimit, bufLimit), remaining());
+    int actualRead = strategy.readFromBlock(stream, expectedRead);
+    if (actualRead == -1) {
+      // The Block Stream reached EOF, but we did not expect it to, so the block
+      // might be corrupt.
+      throw new IOException("Expected to read " + expectedRead + " but got EOF"
+          + " from blockGroup " + stream.getBlockID() + " index "
+          + currentStreamIndex() + 1);
+    }
+    return actualRead;
+  }
+
+  /**
+   * Verify that the input stream is open.
+   * @throws IOException if the connection is closed.
+   */
+  private void checkOpen() throws IOException {
+    if (closed) {
+      throw new IOException(
+          ": " + FSExceptionMessages.STREAM_IS_CLOSED + " Block: "
+              + blockInfo.getBlockID());
+    }
+  }
+
+  @Override
+  public synchronized void close() {
+    closeStreams();
+    closed = true;
+  }
+
+  protected synchronized void closeStreams() {
+    for (int i = 0; i < blockStreams.length; i++) {
+      if (blockStreams[i] != null) {
+        try {
+          blockStreams[i].close();
+          blockStreams[i] = null;
+        } catch (IOException e) {
+          LOG.error("Failed to close stream {}", blockStreams[i], e);
+        }
+      }
+    }
+    // If the streams have been closed outside of a close() call, then it may
+    // be due to freeing resources. If they are reopened, then we will need to
+    // seek the stream to its expected position when the next read is attempted.
+    seeked = true;
+  }
+
+  @Override
+  public synchronized void unbuffer() {
+    for (BlockExtendedInputStream stream : blockStreams) {
+      if (stream != null) {
+        stream.unbuffer();
+      }
+    }
+  }
+
+  @Override
+  public synchronized void seek(long pos) throws IOException {
+    checkOpen();
+    if (pos < 0 || pos >= getLength()) {
+      if (pos == 0) {
+        // It is possible for length and pos to be zero in which case
+        // seek should return instead of throwing exception
+        return;
+      }
+      throw new EOFException(
+          "EOF encountered at pos: " + pos + " for block: "
+              + blockInfo.getBlockID());
+    }
+    position = pos;
+    seeked = true;
+  }
+
+  @Override
+  public synchronized long getPos() {
+    return position;
+  }
+
+  protected synchronized void setPos(long pos) {
+    position = pos;
+  }
+
+  @Override
+  public synchronized boolean seekToNewSource(long l) throws IOException {
+    return false;
+  }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
new file mode 100644
index 0000000..6c39e93
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
@@ -0,0 +1,58 @@
+/*
+ * 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.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Interface used by factories which create ECBlockInput streams for
+ * reconstruction or non-reconstruction reads.
+ */
+public interface ECBlockInputStreamFactory {
+
+  /**
+   * Create a new EC InputStream based on the missingLocations boolean. If it is
+   * set to false, it indicates all locations are available and an
+   * ECBlockInputStream will be created. Otherwise an
+   * ECBlockReconstructedInputStream will be created.
+   * @param missingLocations Indicates if all the data locations are available
+   *                         or not, controlling the type of stream created
+   * @param failedLocations List of DatanodeDetails indicating locations we
+   *                        know are bad and should not be used.
+   * @param repConfig The replication Config
+   * @param blockInfo The blockInfo representing the block.
+   * @param verifyChecksum Whether to verify checksums or not.
+   * @param xceiverFactory Factory to create the xceiver in the client
+   * @param refreshFunction Function to refresh the pipeline if needed
+   * @return BlockExtendedInputStream of the correct type.
+   */
+  BlockExtendedInputStream create(boolean missingLocations,
+      List<DatanodeDetails> failedLocations, ReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      XceiverClientFactory xceiverFactory,
+      Function<BlockID, Pipeline> refreshFunction);
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java
new file mode 100644
index 0000000..470df0c
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java
@@ -0,0 +1,100 @@
+/*
+ * 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.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Factory class to create various BlockStream instances.
+ */
+public final class ECBlockInputStreamFactoryImpl implements
+    ECBlockInputStreamFactory {
+
+  private final BlockInputStreamFactory inputStreamFactory;
+  private final ByteBufferPool byteBufferPool;
+  private final Supplier<ExecutorService> ecReconstructExecutorSupplier;
+
+  public static ECBlockInputStreamFactory getInstance(
+      BlockInputStreamFactory streamFactory, ByteBufferPool byteBufferPool,
+      Supplier<ExecutorService> ecReconstructExecutorSupplier) {
+    return new ECBlockInputStreamFactoryImpl(streamFactory, byteBufferPool,
+        ecReconstructExecutorSupplier);
+  }
+
+  private ECBlockInputStreamFactoryImpl(BlockInputStreamFactory streamFactory,
+      ByteBufferPool byteBufferPool,
+      Supplier<ExecutorService> ecReconstructExecutorSupplier) {
+    this.byteBufferPool = byteBufferPool;
+    this.inputStreamFactory = streamFactory;
+    this.ecReconstructExecutorSupplier = ecReconstructExecutorSupplier;
+  }
+
+  /**
+   * Create a new EC InputStream based on the missingLocations boolean. If it is
+   * set to false, it indicates all locations are available and an
+   * ECBlockInputStream will be created. Otherwise an
+   * ECBlockReconstructedInputStream will be created.
+   * @param missingLocations Indicates if all the data locations are available
+   *                         or not, controlling the type of stream created
+   * @param failedLocations List of DatanodeDetails indicating locations we
+   *                        know are bad and should not be used.
+   * @param repConfig The replication Config
+   * @param blockInfo The blockInfo representing the block.
+   * @param verifyChecksum Whether to verify checksums or not.
+   * @param xceiverFactory Factory to create the xceiver in the client
+   * @param refreshFunction Function to refresh the pipeline if needed
+   * @return BlockExtendedInputStream of the correct type.
+   */
+  public BlockExtendedInputStream create(boolean missingLocations,
+      List<DatanodeDetails> failedLocations, ReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      XceiverClientFactory xceiverFactory,
+      Function<BlockID, Pipeline> refreshFunction) {
+    if (missingLocations) {
+      // We create the reconstruction reader
+      ECBlockReconstructedStripeInputStream sis =
+          new ECBlockReconstructedStripeInputStream(
+              (ECReplicationConfig)repConfig, blockInfo, verifyChecksum,
+              xceiverFactory, refreshFunction, inputStreamFactory,
+              byteBufferPool, ecReconstructExecutorSupplier.get());
+      if (failedLocations != null) {
+        sis.addFailedDatanodes(failedLocations);
+      }
+      return new ECBlockReconstructedInputStream(
+          (ECReplicationConfig) repConfig, byteBufferPool, sis);
+    } else {
+      // Otherwise create the more efficient non-reconstruction reader
+      return new ECBlockInputStream((ECReplicationConfig)repConfig, blockInfo,
+          verifyChecksum, xceiverFactory, refreshFunction, inputStreamFactory);
+    }
+  }
+
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
new file mode 100644
index 0000000..ecde9c6
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
@@ -0,0 +1,239 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Top level class used to read data from EC Encoded blocks. This class decides,
+ * based on the block availability, whether to use a reconstruction or non
+ * reconstruction read and also handles errors from the non-reconstruction reads
+ * failing over to a reconstruction read when they happen.
+ */
+public class ECBlockInputStreamProxy extends BlockExtendedInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockInputStreamProxy.class);
+
+  private final ECReplicationConfig repConfig;
+  private final boolean verifyChecksum;
+  private final XceiverClientFactory xceiverClientFactory;
+  private final Function<BlockID, Pipeline> refreshFunction;
+  private final OmKeyLocationInfo blockInfo;
+  private final ECBlockInputStreamFactory ecBlockInputStreamFactory;
+
+  private BlockExtendedInputStream blockReader;
+  private boolean reconstructionReader = false;
+  private List<DatanodeDetails> failedLocations = new ArrayList<>();
+  private boolean closed = false;
+
+  /**
+   * Given the ECReplicationConfig and the block length, calculate how many
+   * data locations the block should have.
+   * @param repConfig The EC Replication Config
+   * @param blockLength The length of the data block in bytes
+   * @return The number of expected data locations
+   */
+  public static int expectedDataLocations(ECReplicationConfig repConfig,
+      long blockLength) {
+    return (int)Math.min(
+        Math.ceil((double)blockLength / repConfig.getEcChunkSize()),
+        repConfig.getData());
+  }
+
+  /**
+   * From the Pipeline and expected number of locations, determine the number
+   * of data locations available.
+   * @param pipeline The pipeline for the data block, givings its locations and
+   *                 the index of each location.
+   * @param expectedLocs The number of locations we expect for the block to have
+   *                     based on its block length and replication config. The
+   *                     max value should be the repConfig data number.
+   * @return The number of locations available
+   */
+  public static int availableDataLocations(Pipeline pipeline,
+      int expectedLocs) {
+    Set<Integer> locations = new HashSet<>();
+    for (DatanodeDetails dn : pipeline.getNodes()) {
+      int index = pipeline.getReplicaIndex(dn);
+      if (index > 0 && index <= expectedLocs) {
+        locations.add(index);
+      }
+    }
+    return locations.size();
+  }
+
+  public ECBlockInputStreamProxy(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, ECBlockInputStreamFactory streamFactory) {
+    this.repConfig = repConfig;
+    this.verifyChecksum = verifyChecksum;
+    this.blockInfo = blockInfo;
+    this.ecBlockInputStreamFactory = streamFactory;
+    this.xceiverClientFactory = xceiverClientFactory;
+    this.refreshFunction = refreshFunction;
+
+    setReaderType();
+    createBlockReader();
+  }
+
+  private synchronized void setReaderType() {
+    int expected = expectedDataLocations(repConfig, getLength());
+    int available = availableDataLocations(blockInfo.getPipeline(), expected);
+    reconstructionReader = available < expected;
+  }
+
+  private void createBlockReader() {
+    blockReader = ecBlockInputStreamFactory.create(reconstructionReader,
+        failedLocations, repConfig, blockInfo, verifyChecksum,
+        xceiverClientFactory, refreshFunction);
+  }
+
+  @Override
+  public synchronized BlockID getBlockID() {
+    return blockInfo.getBlockID();
+  }
+
+  @Override
+  public synchronized long getRemaining() {
+    return blockReader.getRemaining();
+  }
+
+  @Override
+  public synchronized long getLength() {
+    return blockInfo.getLength();
+  }
+
+  @Override
+  public synchronized int read(byte[] b, int off, int len)
+      throws IOException {
+    return read(ByteBuffer.wrap(b, off, len));
+  }
+
+  @Override
+  public synchronized int read(ByteBuffer buf) throws IOException {
+    ensureNotClosed();
+    if (blockReader.getRemaining() == 0) {
+      return EOF;
+    }
+    int totalRead = 0;
+    long lastPosition = 0;
+    try {
+      while (buf.hasRemaining() && getRemaining() > 0) {
+        buf.mark();
+        lastPosition = blockReader.getPos();
+        totalRead += blockReader.read(buf);
+      }
+    } catch (IOException e) {
+      if (reconstructionReader) {
+        // If we get an error from the reconstruction reader, there
+        // is nothing left to try. It will re-try until it has insufficient
+        // locations internally, so if an error comes here, just re-throw it.
+        throw e;
+      }
+      if (e instanceof BadDataLocationException) {
+        LOG.warn("Failing over to reconstruction read due to an error in " +
+            "ECBlockReader", e);
+        failoverToReconstructionRead(
+            ((BadDataLocationException) e).getFailedLocation(), lastPosition);
+        buf.reset();
+        totalRead += read(buf);
+      } else {
+        throw e;
+      }
+    }
+    return totalRead;
+  }
+
+  private synchronized void failoverToReconstructionRead(
+      DatanodeDetails badLocation, long lastPosition) throws IOException {
+    if (badLocation != null) {
+      failedLocations.add(badLocation);
+    }
+    blockReader.close();
+    reconstructionReader = true;
+    createBlockReader();
+    if (lastPosition != 0) {
+      blockReader.seek(lastPosition);
+    }
+  }
+
+  /**
+   * Should never be called in this class.
+   */
+  @Override
+  protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+      throws IOException {
+    throw new IOException("Not Implemented");
+  }
+
+  @Override
+  public synchronized void unbuffer() {
+    blockReader.unbuffer();
+  }
+
+  @Override
+  public synchronized long getPos() throws IOException {
+    return blockReader != null ? blockReader.getPos() : 0;
+  }
+
+  @Override
+  public synchronized void seek(long pos) throws IOException {
+    ensureNotClosed();
+    try {
+      blockReader.seek(pos);
+    } catch (IOException e) {
+      if (reconstructionReader) {
+        throw e;
+      }
+      failoverToReconstructionRead(null, pos);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (blockReader != null) {
+      blockReader.close();
+    }
+    closed = true;
+  }
+
+  private void ensureNotClosed() throws IOException {
+    if (closed) {
+      throw new IOException("The stream is closed");
+    }
+  }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java
new file mode 100644
index 0000000..ebfa5cc
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java
@@ -0,0 +1,455 @@
+/*
+ * 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.ozone.client.io;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static com.google.common.base.Preconditions.checkState;
+import static org.apache.ratis.util.Preconditions.assertInstanceOf;
+
+/**
+ * ECBlockOutputStreamEntry manages write into EC keys' data block groups.
+ * A block group consists of data and parity blocks. For every block we have
+ * an internal ECBlockOutputStream instance with a single node pipeline, that
+ * is derived from the original EC pipeline.
+ */
+public class ECBlockOutputStreamEntry extends BlockOutputStreamEntry {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockOutputStreamEntry.class);
+
+  private final ECReplicationConfig replicationConfig;
+  private final long length;
+
+  private ECBlockOutputStream[] blockOutputStreams;
+  private int currentStreamIdx = 0;
+  private long successfulBlkGrpAckedLen;
+
+  @SuppressWarnings({"parameternumber", "squid:S00107"})
+  ECBlockOutputStreamEntry(BlockID blockID, String key,
+      XceiverClientFactory xceiverClientManager, Pipeline pipeline, long length,
+      BufferPool bufferPool, Token<OzoneBlockTokenIdentifier> token,
+      OzoneClientConfig config) {
+    super(blockID, key, xceiverClientManager, pipeline, length, bufferPool,
+        token, config);
+    assertInstanceOf(
+        pipeline.getReplicationConfig(), ECReplicationConfig.class);
+    this.replicationConfig =
+        (ECReplicationConfig) pipeline.getReplicationConfig();
+    this.length = replicationConfig.getData() * length;
+  }
+
+  @Override
+  void checkStream() throws IOException {
+    if (!isInitialized()) {
+      blockOutputStreams =
+          new ECBlockOutputStream[replicationConfig.getRequiredNodes()];
+    }
+    if (blockOutputStreams[currentStreamIdx] == null) {
+      createOutputStream();
+    }
+  }
+
+  @Override
+  void createOutputStream() throws IOException {
+    Pipeline ecPipeline = getPipeline();
+    List<DatanodeDetails> nodes = getPipeline().getNodes();
+    blockOutputStreams[currentStreamIdx] = new ECBlockOutputStream(
+        getBlockID(),
+        getXceiverClientManager(),
+        createSingleECBlockPipeline(
+            ecPipeline, nodes.get(currentStreamIdx), currentStreamIdx + 1),
+        getBufferPool(),
+        getConf(),
+        getToken());
+  }
+
+  @Override
+  public OutputStream getOutputStream() {
+    if (!isInitialized()) {
+      return null;
+    }
+    checkState(blockOutputStreams[currentStreamIdx] != null);
+    return blockOutputStreams[currentStreamIdx];
+  }
+
+  @Override
+  boolean isInitialized() {
+    return blockOutputStreams != null;
+  }
+
+  @Override
+  public long getLength() {
+    return length;
+  }
+
+  public int getCurrentStreamIdx() {
+    return currentStreamIdx;
+  }
+
+  public void useNextBlockStream() {
+    currentStreamIdx =
+        (currentStreamIdx + 1) % replicationConfig.getRequiredNodes();
+  }
+
+  public void markFailed(Exception e) {
+    if (blockOutputStreams[currentStreamIdx] != null) {
+      blockOutputStreams[currentStreamIdx].setIoException(e);
+    }
+  }
+
+  public void forceToFirstParityBlock() {
+    currentStreamIdx = replicationConfig.getData();
+  }
+
+  public void resetToFirstEntry() {
+    currentStreamIdx = 0;
+  }
+
+  @Override
+  void incCurrentPosition() {
+    if (isWritingParity()) {
+      return;
+    }
+    super.incCurrentPosition();
+  }
+
+  @Override
+  void incCurrentPosition(long len) {
+    if (isWritingParity()) {
+      return;
+    }
+    super.incCurrentPosition(len);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (!isInitialized()) {
+      return;
+    }
+    for (int i = 0;
+         i <= currentStreamIdx && i < blockOutputStreams.length; i++) {
+      if (blockOutputStreams[i] != null) {
+        blockOutputStreams[i].flush();
+      }
+    }
+  }
+
+  @Override
+  boolean isClosed() {
+    if (!isInitialized()) {
+      return false;
+    }
+    return blockStreams().allMatch(BlockOutputStream::isClosed);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!isInitialized()) {
+      return;
+    }
+    for (ECBlockOutputStream stream : blockOutputStreams) {
+      if (stream != null) {
+        stream.close();
+      }
+    }
+    updateBlockID(underlyingBlockID());
+  }
+
+  @Override
+  long getTotalAckDataLength() {
+    if (!isInitialized()) {
+      return 0;
+    }
+    updateBlockID(underlyingBlockID());
+
+    return this.successfulBlkGrpAckedLen;
+  }
+
+  void updateBlockGroupToAckedPosition(long len) {
+    if (isWritingParity()) {
+      return;
+    }
+    this.successfulBlkGrpAckedLen = len;
+  }
+
+  /**
+   * Returns the amount of bytes that were attempted to be sent through towards
+   * the DataNodes, and the write call succeeded without an exception.
+   * In EC entries the parity writes does not count into this, as the written
+   * data length represents the attempts of the classes using the entry, and
+   * not the attempts of the entry itself.
+   *
+   * @return 0 if the stream is not initialized, the amount of data bytes that
+   *    were attempted to be written to the entry.
+   */
+  //TODO: this might become problematic, and should be tested during the
+  //      implementation of retries and error handling, as if there is a retry,
+  //      then some data might have to be written twice.
+  //      This current implementation is an assumption here.
+  //      We might need to account the parity bytes written here, or elsewhere.
+  @Override
+  long getWrittenDataLength() {
+    if (!isInitialized()) {
+      return 0;
+    }
+    return dataStreams()
+        .mapToLong(BlockOutputStream::getWrittenDataLength)
+        .sum();
+  }
+
+  @Override
+  Collection<DatanodeDetails> getFailedServers() {
+    if (!isInitialized()) {
+      return Collections.emptyList();
+    }
+
+    return blockStreams()
+        .flatMap(outputStream -> outputStream.getFailedServers().stream())
+        .collect(Collectors.toList());
+  }
+
+  @VisibleForTesting
+  Pipeline createSingleECBlockPipeline(Pipeline ecPipeline,
+      DatanodeDetails node, int replicaIndex) {
+    Map<DatanodeDetails, Integer> indiciesForSinglePipeline = new HashMap<>();
+    indiciesForSinglePipeline.put(node, replicaIndex);
+    return Pipeline.newBuilder()
+        .setId(ecPipeline.getId())
+        .setReplicationConfig(ecPipeline.getReplicationConfig())
+        .setState(ecPipeline.getPipelineState())
+        .setNodes(ImmutableList.of(node))
+        .setReplicaIndexes(indiciesForSinglePipeline)
+        .build();
+  }
+
+  void executePutBlock(boolean isClose) {
+    if (!isInitialized()) {
+      return;
+    }
+    for (ECBlockOutputStream stream : blockOutputStreams) {
+      if (stream == null) {
+        continue;
+      }
+      try {
+        stream.executePutBlock(isClose, true);
+      } catch (Exception e) {
+        stream.setIoException(e);
+      }
+    }
+  }
+
+  private BlockID underlyingBlockID() {
+    if (blockOutputStreams[0] == null) {
+      return null;
+    }
+    // blockID is the same for EC blocks inside one block group managed by
+    // this entry, so updating based on the first stream, as when we write any
+    // data that is surely exists.
+    return blockOutputStreams[0].getBlockID();
+  }
+
+  public List<ECBlockOutputStream> streamsWithWriteFailure() {
+    return getFailedStreams(false);
+  }
+
+  public List<ECBlockOutputStream> streamsWithPutBlockFailure() {
+    return getFailedStreams(true);
+  }
+
+  /**
+   * In EC, we will do async write calls for writing data in the scope of a
+   * stripe. After every stripe write finishes, use this method to validate the
+   * responses of current stripe data writes. This method can also be used to
+   * validate the stripe put block responses.
+   * @param forPutBlock If true, it will validate the put block response
+   *                    futures. It will validate stripe data write response
+   *                    futures if false.
+   * @return
+   */
+  private List<ECBlockOutputStream> getFailedStreams(boolean forPutBlock) {
+    final Iterator<ECBlockOutputStream> iter = blockStreams().iterator();
+    List<ECBlockOutputStream> failedStreams = new ArrayList<>();
+    while (iter.hasNext()) {
+      final ECBlockOutputStream stream = iter.next();
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+          responseFuture = null;
+      if (forPutBlock) {
+        responseFuture = stream.getCurrentPutBlkResponseFuture();
+      } else {
+        responseFuture = stream.getCurrentChunkResponseFuture();
+      }
+      if (isFailed(stream, responseFuture)) {
+        failedStreams.add(stream);
+      }
+    }
+    return failedStreams;
+  }
+
+  private boolean isFailed(
+      ECBlockOutputStream outputStream,
+      CompletableFuture<ContainerProtos.
+          ContainerCommandResponseProto> chunkWriteResponseFuture) {
+
+    if (chunkWriteResponseFuture == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Failed to reap response from datanode {}",
+            outputStream.getDatanodeDetails());
+      }
+      return true;
+    }
+
+    ContainerProtos.ContainerCommandResponseProto containerCommandResponseProto
+        = null;
+    try {
+      containerCommandResponseProto = chunkWriteResponseFuture.get();
+    } catch (InterruptedException e) {
+      outputStream.setIoException(e);
+      Thread.currentThread().interrupt();
+    } catch (ExecutionException e) {
+      outputStream.setIoException(e);
+    }
+
+    if (outputStream.getIoException() != null) {
+      return true;
+    }
+
+    if (containerCommandResponseProto == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Empty response from datanode {}",
+            outputStream.getDatanodeDetails());
+      }
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean isWritingParity() {
+    return currentStreamIdx >= replicationConfig.getData();
+  }
+
+  private Stream<ECBlockOutputStream> blockStreams() {
+    return Arrays.stream(blockOutputStreams).filter(Objects::nonNull);
+  }
+
+  private Stream<ECBlockOutputStream> dataStreams() {
+    return Arrays.stream(blockOutputStreams)
+        .limit(replicationConfig.getData())
+        .filter(Objects::nonNull);
+  }
+
+  /**
+   * Builder class for ChunkGroupOutputStreamEntry.
+   * */
+  public static class Builder {
+    private BlockID blockID;
+    private String key;
+    private XceiverClientFactory xceiverClientManager;
+    private Pipeline pipeline;
+    private long length;
+    private BufferPool bufferPool;
+    private Token<OzoneBlockTokenIdentifier> token;
+    private OzoneClientConfig config;
+
+    public ECBlockOutputStreamEntry.Builder setBlockID(BlockID bID) {
+      this.blockID = bID;
+      return this;
+    }
+
+    public ECBlockOutputStreamEntry.Builder setKey(String keys) {
+      this.key = keys;
+      return this;
+    }
+
+    public ECBlockOutputStreamEntry.Builder setXceiverClientManager(
+        XceiverClientFactory
+            xClientManager) {
+      this.xceiverClientManager = xClientManager;
+      return this;
+    }
+
+    public ECBlockOutputStreamEntry.Builder setPipeline(Pipeline ppln) {
+      this.pipeline = ppln;
+      return this;
+    }
+
+    public ECBlockOutputStreamEntry.Builder setLength(long len) {
+      this.length = len;
+      return this;
+    }
+
+    public ECBlockOutputStreamEntry.Builder setBufferPool(BufferPool pool) {
+      this.bufferPool = pool;
+      return this;
+    }
+
+    public ECBlockOutputStreamEntry.Builder setConfig(
+        OzoneClientConfig clientConfig) {
+      this.config = clientConfig;
+      return this;
+    }
+
+    public ECBlockOutputStreamEntry.Builder setToken(
+        Token<OzoneBlockTokenIdentifier> bToken) {
+      this.token = bToken;
+      return this;
+    }
+
+    public ECBlockOutputStreamEntry build() {
+      return new ECBlockOutputStreamEntry(blockID,
+          key,
+          xceiverClientManager,
+          pipeline,
+          length,
+          bufferPool,
+          token, config);
+    }
+  }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java
new file mode 100644
index 0000000..4651896
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java
@@ -0,0 +1,89 @@
+/*
+ * 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.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.ozone.common.MonotonicClock;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+
+import java.time.ZoneOffset;
+
+/**
+ * {@link BlockOutputStreamEntryPool} is responsible to manage OM communication
+ * regarding writing a block to Ozone in a non-EC write case.
+ * The basic operations are fine for us but we need a specific
+ * {@link ECBlockOutputStreamEntry} implementation to handle writing EC block
+ * groups, this class implements the logic that handles the specific EC entries'
+ * instantiation and retrieval from the pool.
+ *
+ * @see ECKeyOutputStream
+ * @see BlockOutputStreamEntryPool
+ * @see ECBlockOutputStreamEntry
+ */
+public class ECBlockOutputStreamEntryPool extends BlockOutputStreamEntryPool {
+
+  @SuppressWarnings({"parameternumber", "squid:S00107"})
+  public ECBlockOutputStreamEntryPool(OzoneClientConfig config,
+      OzoneManagerProtocol omClient,
+      String requestId,
+      ReplicationConfig replicationConfig,
+      String uploadID,
+      int partNumber,
+      boolean isMultipart,
+      OmKeyInfo info,
+      boolean unsafeByteBufferConversion,
+      XceiverClientFactory xceiverClientFactory,
+      long openID) {
+    super(config, omClient, requestId, replicationConfig, uploadID, partNumber,
+        isMultipart, info, unsafeByteBufferConversion, xceiverClientFactory,
+        openID);
+    assert replicationConfig instanceof ECReplicationConfig;
+  }
+
+  @Override
+  ExcludeList createExcludeList() {
+    return new ExcludeList(getConfig().getExcludeNodesExpiryTime(),
+        new MonotonicClock(ZoneOffset.UTC));
+  }
+
+  @Override
+  BlockOutputStreamEntry createStreamEntry(OmKeyLocationInfo subKeyInfo) {
+    return
+        new ECBlockOutputStreamEntry.Builder()
+            .setBlockID(subKeyInfo.getBlockID())
+            .setKey(getKeyName())
+            .setXceiverClientManager(getXceiverClientFactory())
+            .setPipeline(subKeyInfo.getPipeline())
+            .setConfig(getConfig())
+            .setLength(subKeyInfo.getLength())
+            .setBufferPool(getBufferPool())
+            .setToken(subKeyInfo.getToken())
+            .build();
+  }
+
+  @Override
+  public ECBlockOutputStreamEntry getCurrentStreamEntry() {
+    return (ECBlockOutputStreamEntry) super.getCurrentStreamEntry();
+  }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
new file mode 100644
index 0000000..fe93b2e
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
@@ -0,0 +1,243 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.io.ByteBufferPool;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Input stream which wraps a ECBlockReconstructedStripeInputStream to allow
+ * a EC Block to be read via the traditional InputStream read methods.
+ */
+public class ECBlockReconstructedInputStream extends BlockExtendedInputStream {
+
+  private ECReplicationConfig repConfig;
+  private ECBlockReconstructedStripeInputStream stripeReader;
+  private ByteBuffer[] bufs;
+  private final ByteBufferPool byteBufferPool;
+  private boolean closed = false;
+  private boolean unBuffered = false;
+
+  private long position = 0;
+
+  public ECBlockReconstructedInputStream(ECReplicationConfig repConfig,
+      ByteBufferPool byteBufferPool,
+      ECBlockReconstructedStripeInputStream stripeReader) {
+    this.repConfig = repConfig;
+    this.byteBufferPool = byteBufferPool;
+    this.stripeReader = stripeReader;
+  }
+
+  @Override
+  public synchronized BlockID getBlockID() {
+    return stripeReader.getBlockID();
+  }
+
+  @Override
+  public synchronized long getRemaining() {
+    return getLength() - position;
+  }
+
+  @Override
+  public synchronized long getLength() {
+    return stripeReader.getLength();
+  }
+
+  @Override
+  public synchronized int read(byte[] b, int off, int len)
+      throws IOException {
+    return read(ByteBuffer.wrap(b, off, len));
+  }
+
+  @Override
+  public synchronized int read(ByteBuffer buf) throws IOException {
+    ensureNotClosed();
+    if (!hasRemaining()) {
+      return EOF;
+    }
+    allocateBuffers();
+    if (unBuffered) {
+      seek(getPos());
+      unBuffered = false;
+    }
+    int totalRead = 0;
+    while (buf.hasRemaining() && getRemaining() > 0) {
+      ByteBuffer b = selectNextBuffer();
+      if (b == null) {
+        // This should not happen, so if it does abort.
+        throw new IOException(getRemaining() + " bytes remaining but unable " +
+            "to select a buffer with data");
+      }
+      long read = readBufferToDest(b, buf);
+      totalRead += read;
+    }
+    if (!hasRemaining()) {
+      // We have reached the end of the block. While the block is still open
+      // and could be seeked back, it is most likely the block will be closed.
+      // KeyInputStream does not call close on the block until all blocks in the
+      // key have been read, so releasing the resources here helps to avoid
+      // excessive memory usage.
+      freeBuffers();
+    }
+    return totalRead;
+  }
+
+  private void ensureNotClosed() throws IOException {
+    if (closed) {
+      throw new IOException("The input stream is closed");
+    }
+  }
+
+  private ByteBuffer selectNextBuffer() throws IOException {
+    for (ByteBuffer b : bufs) {
+      if (b.hasRemaining()) {
+        return b;
+      }
+    }
+    // If we get here, then no buffer has any remaining, so we need to
+    // fill them.
+    long read = readStripe();
+    if (read == EOF) {
+      return null;
+    }
+    return selectNextBuffer();
+  }
+
+  private long readBufferToDest(ByteBuffer src, ByteBuffer dest) {
+    int initialRemaining = dest.remaining();
+    while (dest.hasRemaining() && src.hasRemaining()) {
+      dest.put(src.get());
+    }
+    int read = initialRemaining - dest.remaining();
+    position += read;
+    return read;
+  }
+
+  @Override
+  protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+      throws IOException {
+    throw new IOException("Not Implemented");
+  }
+
+  @Override
+  public synchronized void unbuffer() {
+    stripeReader.unbuffer();
+    freeBuffers();
+    unBuffered = true;
+  }
+
+  @Override
+  public synchronized long getPos() throws IOException {
+    return position;
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    stripeReader.close();
+    freeBuffers();
+    closed = true;
+  }
+
+  private void freeBuffers() {
+    if (bufs != null) {
+      for (int i = 0; i < bufs.length; i++) {
+        byteBufferPool.putBuffer(bufs[i]);
+        bufs[i] = null;
+      }
+      bufs = null;
+    }
+  }
+
+  @Override
+  public synchronized void seek(long pos) throws IOException {
+    ensureNotClosed();
+    if (pos < 0 || pos >= getLength()) {
+      if (pos == 0) {
+        // It is possible for length and pos to be zero in which case
+        // seek should return instead of throwing exception
+        return;
+      }
+      throw new EOFException(
+          "EOF encountered at pos: " + pos + " for block: " + getBlockID());
+    }
+    long stripeSize = (long)repConfig.getEcChunkSize() * repConfig.getData();
+    long stripeNum = pos / stripeSize;
+    int partial = (int)(pos % stripeSize);
+    // Seek the stripe reader to the beginning of the new current stripe
+    stripeReader.seek(stripeNum * stripeSize);
+    // Now reload the data buffers and adjust their position to the partial
+    // stripe offset.
+    readAndSeekStripe(partial);
+    position = pos;
+  }
+
+  private void readAndSeekStripe(int offset) throws IOException {
+    allocateBuffers();
+    readStripe();
+    if (offset == 0) {
+      return;
+    }
+    for (ByteBuffer b : bufs) {
+      int newPos = Math.min(b.remaining(), offset);
+      b.position(newPos);
+      offset -= newPos;
+      if (offset == 0) {
+        break;
+      }
+    }
+  }
+
+  private long readStripe() throws IOException {
+    clearBuffers();
+    return stripeReader.readStripe(bufs);
+  }
+
+  private void allocateBuffers() {
+    if (bufs != null) {
+      return;
+    }
+    bufs = new ByteBuffer[repConfig.getData()];
+    for (int i = 0; i < repConfig.getData(); i++) {
+      bufs[i] = byteBufferPool.getBuffer(false, repConfig.getEcChunkSize());
+      // Initially set the limit to 0 so there is no remaining space.
+      bufs[i].limit(0);
+    }
+  }
+
+  private void clearBuffers() {
+    for (ByteBuffer b : bufs) {
+      b.clear();
+      // As we are getting buffers from a bufferPool, we may get buffers with a
+      // capacity larger than what we asked for. After calling clear(), the
+      // buffer limit will become the capacity so we need to reset it back to
+      // the desired limit.
+      b.limit(repConfig.getEcChunkSize());
+    }
+  }
+
+  private boolean hasRemaining() {
+    return getRemaining() > 0;
+  }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
new file mode 100644
index 0000000..d5ec6db
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
@@ -0,0 +1,631 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ozone.erasurecode.rawcoder.util.CodecUtil;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+  // Data Indexes we have tried to read from, and failed for some reason
+  private Set<Integer> failedDataIndexes = new HashSet<>();
+  private ByteBufferPool byteBufferPool;
+
+  private RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  private ExecutorService executor;
+
+  @SuppressWarnings("checkstyle:ParameterNumber")
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory,
+      ByteBufferPool byteBufferPool,
+      ExecutorService ecReconstructExecutor) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+    this.byteBufferPool = byteBufferPool;
+    this.executor = ecReconstructExecutor;
+  }
+
+  /**
+   * Provide a list of datanodes that are known to be bad, and no attempt will
+   * be made to read from them. If too many failed nodes are passed, then the
+   * reader may not have sufficient locations available to reconstruct the data.
+   *
+   * Note this call must be made before any attempt it made to read data,
+   * as that is when the reader is initialized. Attempting to call this method
+   * after a read will result in a runtime exception.
+   *
+   * @param dns A list of DatanodeDetails that are known to be bad.
+   */
+  public synchronized void addFailedDatanodes(List<DatanodeDetails> dns) {
+    if (initialized) {
+      throw new RuntimeException("Cannot add failed datanodes after the " +
+          "reader has been initialized");
+    }
+    DatanodeDetails[] locations = getDataLocations();
+    for (DatanodeDetails dn : dns) {
+      for (int i = 0; i < locations.length; i++) {
+        if (locations[i] != null && locations[i].equals(dn)) {
+          failedDataIndexes.add(i);
+          break;
+        }
+      }
+    }
+  }
+
+  private void init() throws InsufficientLocationsException {
+    if (decoder == null) {
+      decoder = CodecUtil.createRawDecoderWithFallback(getRepConfig());
+    }
+    if (decoderInputBuffers == null) {
+      // The EC decoder needs an array data+parity long, with missing or not
+      // needed indexes set to null.
+      decoderInputBuffers = new ByteBuffer[getRepConfig().getRequiredNodes()];
+    }
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are insufficient " +
+          "datanodes to read the EC block");
+    }
+    dataIndexes.clear();
+    ECReplicationConfig repConfig = getRepConfig();
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now. On re-init, we reuse any parity buffers
+    // already allocated.
+    for (int i = repConfig.getData(); i < repConfig.getRequiredNodes(); i++) {
+      if (parityIndexes.contains(i)) {
+        if (decoderInputBuffers[i] == null) {
+          decoderInputBuffers[i] = allocateBuffer(repConfig);
+        }
+      } else {
+        decoderInputBuffers[i] = null;
+      }
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if ((locations[i] == null || failedDataIndexes.contains(i))
+          && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null && !failedDataIndexes.contains(i)) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+        decoderInputBuffers[i] = null;
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    if (!initialized) {
+      init();
+    }
+    validateBuffers(bufs);
+    while (true) {
+      try {
+        assignBuffers(bufs);
+        clearParityBuffers();
+        // Set the read limits on the buffers so we do not read any garbage data
+        // from the end of the block that is unexpected.
+        setBufferReadLimits(bufs, toRead);
+        loadDataBuffersFromStream();
+        break;
+      } catch (IOException e) {
+        // Re-init now the bad block has been excluded. If we have ran out of
+        // locations, init will throw an InsufficientLocations exception.
+        init();
+        // seek to the current position so it rewinds any blocks we read
+        // already.
+        seek(getPos());
+        // Reset the input positions back to zero
+        for (ByteBuffer b : bufs) {
+          b.position(0);
+        }
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        throw new IOException("Interrupted waiting for reads to complete", ie);
+      }
+    }
+    if (missingIndexes.length > 0) {
+      padBuffers(toRead);
+      flipInputs();
+      decodeStripe();
+      // Reset the buffer positions and limits to remove any padding added
+      // before EC Decode.
+      setBufferReadLimits(bufs, toRead);
+    } else {
+      // If we have no missing indexes, then the buffers will be at their
+      // limits after reading so we need to flip them to ensure they are ready
+      // to read by the caller.
+      flipInputs();
+    }
+    setPos(getPos() + toRead);
+    if (remaining() == 0) {
+      // If we reach the end of the block (ie remaining is zero) we free
+      // the underlying streams and buffers. This is because KeyInputStream,
+      // which reads from the EC streams does not close the blocks until it has
+      // read all blocks in the key.
+      freeAllResourcesWithoutClosing();
+    }
+    return toRead;
+  }
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    if (toRead >= getStripeSize()) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    int fullChunks = toRead / chunkSize;
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void setBufferReadLimits(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0) {
+      bufs[0].limit(toRead);
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+      // If we have less than 1 chunk, then the parity buffers are the size
+      // of the first chunk.
+      for (int i = getRepConfig().getData();
+           i < getRepConfig().getRequiredNodes(); i++) {
+        ByteBuffer b = decoderInputBuffers[i];
+        if (b != null) {
+          b.limit(toRead);
+        }
+      }
+    } else {
+      int remainingLength = toRead % chunkSize;
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are already used, and the others which are
+   * available, and select random indexes to meet numRequired. The resulting
+   * list is sorted in ascending order of the indexes.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    List<Integer> selected = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData(); i < repConfig.getRequiredNodes(); i++) {
+      if (locations[i] != null && !failedDataIndexes.contains(i)
+          && decoderInputBuffers[i] == null) {
+        indexes.add(i);
+      }
+      // If we are re-initializing, we want to make sure we are re-using any
+      // previously selected good parity indexes, as the block stream is already
+      // opened.
+      if (decoderInputBuffers[i] != null && !failedDataIndexes.contains(i)) {
+        selected.add(i);
+      }
+    }
+    Preconditions.assertTrue(indexes.size() + selected.size() >= numRequired);
+    Random rand = new Random();
+    while (selected.size() < numRequired) {
+      selected.add(indexes.remove(rand.nextInt(indexes.size())));
+    }
+    Collections.sort(selected);
+    return selected;
+  }
+
+  private ByteBuffer allocateBuffer(ECReplicationConfig repConfig) {
+    ByteBuffer buf = byteBufferPool.getBuffer(
+        false, repConfig.getEcChunkSize());
+    return buf;
+  }
+
+  private void flipInputs() {
+    for (ByteBuffer b : decoderInputBuffers) {
+      if (b != null) {
+        b.flip();
+      }
+    }
+  }
+
+  private void clearParityBuffers() {
+    for (int i = getRepConfig().getData();
+         i < getRepConfig().getRequiredNodes(); i++) {
+      if (decoderInputBuffers[i] != null) {
+        decoderInputBuffers[i].clear();
+      }
+    }
+  }
+
+  protected void loadDataBuffersFromStream()
+      throws IOException, InterruptedException {
+    Queue<ImmutablePair<Integer, Future<Void>>> pendingReads
+        = new ArrayDeque<>();
+    for (int i : dataIndexes) {
+      pendingReads.add(new ImmutablePair<>(i, executor.submit(() -> {
+        readIntoBuffer(i, decoderInputBuffers[i]);
+        return null;
+      })));
+    }
+    boolean exceptionOccurred = false;
+    while (!pendingReads.isEmpty()) {
+      int index = -1;
+      try {
+        ImmutablePair<Integer, Future<Void>> pair = pendingReads.poll();
+        index = pair.getKey();
+        // Should this future.get() have a timeout? At the end of the call chain
+        // we eventually call a grpc or ratis client to read the block data. Its
+        // the call to the DNs which could potentially block. There is a timeout
+        // on that call controlled by:
+        //     OZONE_CLIENT_READ_TIMEOUT = "ozone.client.read.timeout";
+        // Which defaults to 30s. So if there is a DN communication problem, it
+        // should timeout in the client which should propagate up the stack as
+        // an IOException.
+        pair.getValue().get();
+      } catch (ExecutionException ee) {
+        LOG.warn("Failed to read from block {} EC index {}. Excluding the " +
+            "block", getBlockID(), index + 1, ee.getCause());
+        failedDataIndexes.add(index);
+        exceptionOccurred = true;
+      } catch (InterruptedException ie) {
+        // Catch each InterruptedException to ensure all the futures have been
+        // handled, and then throw the exception later
+        LOG.debug("Interrupted while waiting for reads to complete", ie);
+        Thread.currentThread().interrupt();
+      }
+    }
+    if (Thread.currentThread().isInterrupted()) {
+      throw new InterruptedException(
+          "Interrupted while waiting for reads to complete");
+    }
+    if (exceptionOccurred) {
+      throw new IOException("One or more errors occurred reading blocks");
+    }
+  }
+
+  private void readIntoBuffer(int ind, ByteBuffer buf) throws IOException {
+    BlockExtendedInputStream stream = getOrOpenStream(ind);
+    seekStreamIfNecessary(stream, 0);
+    while (buf.hasRemaining()) {
+      int read = stream.read(buf);
+      if (read == EOF) {
+        // We should not reach EOF, as the block should have enough data to
+        // fill the buffer. If the block does not, then it indicates the
+        // block is not as long as it should be, based on the block length
+        // stored in OM. Therefore if there is any remaining space in the
+        // buffer, we should throw an exception.
+        if (buf.hasRemaining()) {
+          throw new IOException("Expected to read " + buf.remaining() +
+              " bytes from block " + getBlockID() + " EC index " + (ind + 1) +
+              " but reached EOF");
+        }
+        break;
+      }
+    }
+  }
+
+  /**
+   * Take the populated input buffers and missing indexes and create the
+   * outputs. Note that the input buffers have to be "ready for read", ie they
+   * need to have been flipped after their data was loaded. The created outputs
+   * are "ready to read" by the underlying decoder API, so there is no need to
+   * flip them after the call. The decoder reads all the inputs leaving the
+   * buffer position at the end, so the inputs are flipped after the decode so
+   * we have a complete set of "outputs" for the EC Stripe which are ready to
+   * read.
+   * @throws IOException
+   */
+  private void decodeStripe() throws IOException {
+    decoder.decode(decoderInputBuffers, missingIndexes, decoderOutputBuffers);
+    flipInputs();
+  }
+
+  @Override
+  public synchronized boolean hasSufficientLocations() {
+    // The number of locations needed is a function of the EC Chunk size. If the
+    // block length is <= the chunk size, we should only have one data location.
+    // If it is greater than the chunk size but less than chunk_size * 2, then
+    // we must have two locations. If it is greater than chunk_size * data_num,
+    // then we must have all data_num locations.
+    // The remaining data locations (for small block lengths) can be assumed to
+    // be all zeros.
+    // Then we need a total of dataNum blocks available across the available
+    // data, parity and padding blocks.
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    int availableLocations =
+        availableDataLocations(expectedDataBlocks) + availableParityLocations();
+    int paddedLocations = repConfig.getData() - expectedDataBlocks;
+    int failedLocations = failedDataIndexes.size();
+
+    if (availableLocations + paddedLocations - failedLocations
+        >= repConfig.getData()) {
+      return true;
+    } else {
+      LOG.error("There are insufficient locations. {} available; {} padded;" +
+          " {} failed; {} expected;", availableLocations, paddedLocations,
+          failedLocations, expectedDataBlocks);
+      return false;
+    }
+  }
+
+  @Override
+  protected int readWithStrategy(ByteReaderStrategy strategy) {
+    throw new NotImplementedException("readWithStrategy is not implemented. " +
+        "Use readStripe() instead");
+  }
+
+  @Override
+  public synchronized void close() {
+    super.close();
+    freeBuffers();
+  }
+
+  @Override
+  public synchronized void unbuffer() {
+    super.unbuffer();
+    freeBuffers();
+  }
+
+  private void freeBuffers() {
+    // Inside this class, we only allocate buffers to read parity into. Data
+    // is reconstructed or read into a set of buffers passed in from the calling
+    // class. Therefore we only need to ensure we free the parity buffers here.
+    if (decoderInputBuffers != null) {
+      for (int i = getRepConfig().getData();
+           i < getRepConfig().getRequiredNodes(); i++) {
+        ByteBuffer buf = decoderInputBuffers[i];
+        if (buf != null) {
+          byteBufferPool.putBuffer(buf);
+          decoderInputBuffers[i] = null;
+        }
+      }
+    }
+    initialized = false;
+  }
+
+  private void freeAllResourcesWithoutClosing() throws IOException {
+    LOG.debug("Freeing all resources while leaving the block open");
+    freeBuffers();
+    closeStreams();
+  }
+
+  @Override
+  public synchronized void seek(long pos) throws IOException {
+    if (pos % getStripeSize() != 0) {
+      // As this reader can only return full stripes, we only seek to the start
+      // stripe offsets
+      throw new IOException("Requested position " + pos
+          + " does not align with a stripe offset");
+    }
+    super.seek(pos);
+  }
+
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java
new file mode 100644
index 0000000..2c51962
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java
@@ -0,0 +1,706 @@
+/*
+ * 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.ozone.client.io;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.ozone.erasurecode.rawcoder.util.CodecUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ECKeyOutputStream handles the EC writes by writing the data into underlying
+ * block output streams chunk by chunk.
+ */
+public final class ECKeyOutputStream extends KeyOutputStream {
+  private OzoneClientConfig config;
+  private ECChunkBuffers ecChunkBufferCache;
+  private int ecChunkSize;
+  private final int numDataBlks;
+  private final int numParityBlks;
+  private final ByteBufferPool bufferPool;
+  private final RawErasureEncoder encoder;
+
+  private enum StripeWriteStatus {
+    SUCCESS,
+    FAILED
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(KeyOutputStream.class);
+
+  private boolean closed;
+  // how much of data is actually written yet to underlying stream
+  private long offset;
+  // how much data has been ingested into the stream
+  private long writeOffset;
+  private final ECBlockOutputStreamEntryPool blockOutputStreamEntryPool;
+
+  @VisibleForTesting
+  public List<BlockOutputStreamEntry> getStreamEntries() {
+    return blockOutputStreamEntryPool.getStreamEntries();
+  }
+
+  @VisibleForTesting
+  public XceiverClientFactory getXceiverClientFactory() {
+    return blockOutputStreamEntryPool.getXceiverClientFactory();
+  }
+
+  @VisibleForTesting
+  public List<OmKeyLocationInfo> getLocationInfoList() {
+    return blockOutputStreamEntryPool.getLocationInfoList();
+  }
+
+  private ECKeyOutputStream(Builder builder) {
+    this.config = builder.getClientConfig();
+    this.bufferPool = builder.getByteBufferPool();
+    // For EC, cell/chunk size and buffer size can be same for now.
+    ecChunkSize = builder.getReplicationConfig().getEcChunkSize();
+    this.config.setStreamBufferMaxSize(ecChunkSize);
+    this.config.setStreamBufferFlushSize(ecChunkSize);
+    this.config.setStreamBufferSize(ecChunkSize);
+    this.numDataBlks = builder.getReplicationConfig().getData();
+    this.numParityBlks = builder.getReplicationConfig().getParity();
+    ecChunkBufferCache = new ECChunkBuffers(
+        ecChunkSize, numDataBlks, numParityBlks, bufferPool);
+    OmKeyInfo info = builder.getOpenHandler().getKeyInfo();
+    blockOutputStreamEntryPool =
+        new ECBlockOutputStreamEntryPool(config,
+            builder.getOmClient(), builder.getRequestID(),
+            builder.getReplicationConfig(),
+            builder.getMultipartUploadID(), builder.getMultipartNumber(),
+            builder.isMultipartKey(),
+            info, builder.isUnsafeByteBufferConversionEnabled(),
+            builder.getXceiverManager(), builder.getOpenHandler().getId());
+
+    this.writeOffset = 0;
+    this.encoder = CodecUtil.createRawEncoderWithFallback(
+        builder.getReplicationConfig());
+  }
+
+  /**
+   * When a key is opened, it is possible that there are some blocks already
+   * allocated to it for this open session. In this case, to make use of these
+   * blocks, we need to add these blocks to stream entries. But, a key's version
+   * also includes blocks from previous versions, we need to avoid adding these
+   * old blocks to stream entries, because these old blocks should not be picked
+   * for write. To do this, the following method checks that, only those
+   * blocks created in this particular open version are added to stream entries.
+   *
+   * @param version     the set of blocks that are pre-allocated.
+   * @param openVersion the version corresponding to the pre-allocation.
+   * @throws IOException
+   */
+  public void addPreallocateBlocks(OmKeyLocationInfoGroup version,
+      long openVersion) throws IOException {
+    blockOutputStreamEntryPool.addPreallocateBlocks(version, openVersion);
+  }
+
+  /**
+   * Try to write the bytes sequence b[off:off+len) to underlying EC block
+   * streams.
+   *
+   * @param b   byte data
+   * @param off starting offset
+   * @param len length to write
+   * @throws IOException
+   */
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    checkNotClosed();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length)
+        || ((off + len) < 0)) {
+      throw new IndexOutOfBoundsException();
+    }
+    int rem = len;
+    while (rem > 0) {
+      try {
+        blockOutputStreamEntryPool.allocateBlockIfNeeded();
+        int currentStreamIdx = blockOutputStreamEntryPool
+            .getCurrentStreamEntry().getCurrentStreamIdx();
+        int bufferRem =
+            ecChunkBufferCache.dataBuffers[currentStreamIdx].remaining();
+        int writeLen = Math.min(rem, Math.min(bufferRem, ecChunkSize));
+        int pos = handleDataWrite(currentStreamIdx, b, off, writeLen);
+        checkAndWriteParityCells(pos);
+        rem -= writeLen;
+        off += writeLen;
+      } catch (Exception e) {
+        markStreamClosed();
+        throw new IOException(e.getMessage());
+      }
+    }
+    writeOffset += len;
+  }
+
+  private StripeWriteStatus rewriteStripeToNewBlockGroup(
+      long failedStripeDataSize, boolean close) throws IOException {
+    int[] failedDataStripeChunkLens = new int[numDataBlks];
+    int[] failedParityStripeChunkLens = new int[numParityBlks];
+    final ByteBuffer[] dataBuffers = ecChunkBufferCache.getDataBuffers();
+    for (int i = 0; i < numDataBlks; i++) {
+      failedDataStripeChunkLens[i] = dataBuffers[i].limit();
+    }
+    final ByteBuffer[] parityBuffers = ecChunkBufferCache.getParityBuffers();
+    for (int i = 0; i < numParityBlks; i++) {
+      failedParityStripeChunkLens[i] = parityBuffers[i].limit();
+    }
+
+    // Rollback the length/offset updated as part of this failed stripe write.
+    offset -= failedStripeDataSize;
+
+    final ECBlockOutputStreamEntry failedStreamEntry =
+        blockOutputStreamEntryPool.getCurrentStreamEntry();
+    failedStreamEntry.resetToFirstEntry();
+    failedStreamEntry.resetToAckedPosition();
+    // All pre-allocated blocks from the same pipeline
+    // should be dropped to eliminate worthless retries.
+    blockOutputStreamEntryPool.discardPreallocatedBlocks(-1,
+        failedStreamEntry.getPipeline().getId());
+    // Let's close the current entry.
+    failedStreamEntry.close();
+
+    // Let's rewrite the last stripe, so that it will be written to new block
+    // group.
+    // TODO: we can improve to write partial stripe failures. In that case,
+    //  we just need to write only available buffers.
+    blockOutputStreamEntryPool.allocateBlockIfNeeded();
+    final ECBlockOutputStreamEntry currentStreamEntry =
+        blockOutputStreamEntryPool.getCurrentStreamEntry();
+    long totalLenToWrite = failedStripeDataSize;
+    for (int i = 0; i < numDataBlks; i++) {
+      int currentLen = (int) (totalLenToWrite < failedDataStripeChunkLens[i] ?
+          totalLenToWrite :
+          failedDataStripeChunkLens[i]);
+      if (currentLen > 0) {
+        handleOutputStreamWrite(i, currentLen, false);
+      }
+      currentStreamEntry.useNextBlockStream();
+      totalLenToWrite -= currentLen;
+    }
+    for (int i = 0; i < (numParityBlks); i++) {
+      handleOutputStreamWrite(i + numDataBlks, failedParityStripeChunkLens[i],
+          true);
+      currentStreamEntry.useNextBlockStream();
+    }
+
+    if (hasWriteFailure()) {
+      handleFailedStreams(false);
+      return StripeWriteStatus.FAILED;
+    }
+    currentStreamEntry.executePutBlock(close);
+
+    if (hasPutBlockFailure()) {
+      handleFailedStreams(true);
+      return StripeWriteStatus.FAILED;
+    }
+    ECBlockOutputStreamEntry newBlockGroupStreamEntry =
+        blockOutputStreamEntryPool.getCurrentStreamEntry();
+    newBlockGroupStreamEntry
+        .updateBlockGroupToAckedPosition(failedStripeDataSize);
+    ecChunkBufferCache.clear();
+
+    if (newBlockGroupStreamEntry.getRemaining() <= 0) {
+      // In most cases this should not happen except in the case stripe size and
+      // block size same.
+      newBlockGroupStreamEntry.close();
+    } else {
+      newBlockGroupStreamEntry.resetToFirstEntry();
+    }
+
+    return StripeWriteStatus.SUCCESS;
+  }
+
+  private void checkAndWriteParityCells(int lastDataBuffPos)
+      throws IOException {
+    // Check data blocks finished
+    // If index > numDataBlks
+    ECBlockOutputStreamEntry currentStreamEntry =
+        blockOutputStreamEntryPool.getCurrentStreamEntry();
+    int currentStreamIdx = currentStreamEntry.getCurrentStreamIdx();
+    if (currentStreamIdx == numDataBlks && lastDataBuffPos == ecChunkSize) {
+      //Lets encode and write
+      boolean shouldClose = currentStreamEntry.getRemaining() <= 0;
+      if (handleParityWrites(ecChunkSize, shouldClose)
+          == StripeWriteStatus.FAILED) {
+        handleStripeFailure(numDataBlks * ecChunkSize, shouldClose);
+      } else {
+        // At this stage stripe write is successful.
+        currentStreamEntry.updateBlockGroupToAckedPosition(
+            currentStreamEntry.getCurrentPosition());
+      }
+
+    }
+  }
+
+  private StripeWriteStatus handleParityWrites(int parityCellSize,
+      boolean isLastStripe) throws IOException {
+    writeParityCells(parityCellSize);
+    if (hasWriteFailure()) {
+      handleFailedStreams(false);
+      return StripeWriteStatus.FAILED;
+    }
+
+    // By this time, we should have finished full stripe. So, lets call
+    // executePutBlock for all.
+    // TODO: we should alter the put block calls to share CRC to each stream.
+    ECBlockOutputStreamEntry streamEntry =
+        blockOutputStreamEntryPool.getCurrentStreamEntry();
+    streamEntry
+        .executePutBlock(isLastStripe);
+
+    if (hasPutBlockFailure()) {
+      handleFailedStreams(true);
+      return StripeWriteStatus.FAILED;
+    }
+    ecChunkBufferCache.clear();
+
+    if (streamEntry.getRemaining() <= 0) {
+      streamEntry.close();
+    } else {
+      streamEntry.resetToFirstEntry();
+    }
+
+    return StripeWriteStatus.SUCCESS;
+  }
+
+  private boolean hasWriteFailure() {
+    return !blockOutputStreamEntryPool.getCurrentStreamEntry()
+        .streamsWithWriteFailure().isEmpty();
+  }
+
+  private boolean hasPutBlockFailure() {
+    return !blockOutputStreamEntryPool.getCurrentStreamEntry()
+        .streamsWithPutBlockFailure().isEmpty();
+  }
+
+  private void handleFailedStreams(boolean forPutBlock) {
+    ECBlockOutputStreamEntry currentStreamEntry =
+        blockOutputStreamEntryPool.getCurrentStreamEntry();
+    List<ECBlockOutputStream> failedStreams = forPutBlock
+        ? currentStreamEntry.streamsWithPutBlockFailure()
+        : currentStreamEntry.streamsWithWriteFailure();
+
+    // Since writes are async, let's check the failures once.
+    boolean containerToExcludeAll = true;
+    for (ECBlockOutputStream failedStream : failedStreams) {
+      Throwable cause = HddsClientUtils.checkForException(
+          failedStream.getIoException());
+      Preconditions.checkNotNull(cause);
+      if (!checkIfContainerToExclude(cause)) {
+        blockOutputStreamEntryPool.getExcludeList()
+            .addDatanode(failedStream.getDatanodeDetails());
+        containerToExcludeAll = false;
+      }
+    }
+
+    // NOTE: For now, this is mainly for ContainerNotOpenException
+    // due to container full, but may also for those cases that
+    // a DN do respond but with one with certain failures.
+    // In such cases we don't treat the replied DNs as failed.
+    if (containerToExcludeAll) {
+      blockOutputStreamEntryPool.getExcludeList()
+          .addPipeline(currentStreamEntry.getPipeline().getId());
+    }
+  }
+
+  @Override
+  protected boolean checkIfContainerToExclude(Throwable t) {
+    return super.checkIfContainerToExclude(t)
+        && t instanceof ContainerNotOpenException;
+  }
+
+  void writeParityCells(int parityCellSize) throws IOException {
+    final ByteBuffer[] buffers = ecChunkBufferCache.getDataBuffers();
+    final ByteBuffer[] parityBuffers = ecChunkBufferCache.getParityBuffers();
+
+    for (ByteBuffer b : parityBuffers) {
+      b.limit(parityCellSize);
+    }
+    for (ByteBuffer b : buffers) {
+      b.flip();
+    }
+    encoder.encode(buffers, parityBuffers);
+    blockOutputStreamEntryPool
+        .getCurrentStreamEntry().forceToFirstParityBlock();
+    for (int i =
+         numDataBlks; i < (this.numDataBlks + this.numParityBlks); i++) {
+      // Move the stream entry cursor to parity block index
+      handleParityWrite(i, parityCellSize);
+    }
+  }
+
+  private int handleDataWrite(int currIdx, byte[] b, int off, int len) {
+    int pos = ecChunkBufferCache.addToDataBuffer(currIdx, b, off, len);
+    if (pos == ecChunkSize) {
+      handleOutputStreamWrite(currIdx, pos, false);
+      blockOutputStreamEntryPool.getCurrentStreamEntry().useNextBlockStream();
+    }
+    return pos;
+  }
+
+  private void handleParityWrite(int currIdx, int len) {
+    handleOutputStreamWrite(currIdx, len, true);
+    blockOutputStreamEntryPool.getCurrentStreamEntry().useNextBlockStream();
+  }
+
+  private void handleOutputStreamWrite(int currIdx, int len, boolean isParity) {
+    ByteBuffer bytesToWrite = isParity ?
+        ecChunkBufferCache.getParityBuffers()[currIdx - numDataBlks] :
+        ecChunkBufferCache.getDataBuffers()[currIdx];
+    try {
+      // Since it's a full cell, let's write all content from buffer.
+      // At a time we write max cell size in EC. So, it should safe to cast
+      // the len to int to use the super class defined write API.
+      // The len cannot be bigger than cell buffer size.
+      assert len <= ecChunkSize : " The len: " + len + ". EC chunk size: "
+          + ecChunkSize;
+      assert len <= bytesToWrite
+          .limit() : " The len: " + len + ". Chunk buffer limit: "
+          + bytesToWrite.limit();
+      writeToOutputStream(blockOutputStreamEntryPool.getCurrentStreamEntry(),
+          bytesToWrite.array(), len, 0, isParity);
+    } catch (Exception e) {
+      markStreamAsFailed(e);
+    }
+  }
+
+  private long writeToOutputStream(ECBlockOutputStreamEntry current,
+      byte[] b, int writeLen, int off, boolean isParity)
+      throws IOException {
+    try {
+      if (!isParity) {
+        // In case if exception while writing, this length will be updated back
+        // as part of handleStripeFailure.
+        offset += writeLen;
+      }
+      current.write(b, off, writeLen);
+    } catch (IOException ioe) {
+      LOG.debug(
+          "Exception while writing the cell buffers. The writeLen: " + writeLen
+              + ". The block internal index is: "
+              + current
+              .getCurrentStreamIdx(), ioe);
+      handleException(current, ioe);
+    }
+    return writeLen;
+  }
+
+  private void handleException(BlockOutputStreamEntry streamEntry,
+      IOException exception) throws IOException {
+    Throwable t = HddsClientUtils.checkForException(exception);
+    Preconditions.checkNotNull(t);
+    boolean containerExclusionException = checkIfContainerToExclude(t);
+    if (containerExclusionException) {
+      blockOutputStreamEntryPool.getExcludeList()
+          .addPipeline(streamEntry.getPipeline().getId());
+    }
+    markStreamAsFailed(exception);
+  }
+
+  private void markStreamClosed() {
+    blockOutputStreamEntryPool.cleanup();
+    closed = true;
+  }
+
+  private void markStreamAsFailed(Exception e) {
+    blockOutputStreamEntryPool.getCurrentStreamEntry().markFailed(e);
+  }
+
+  @Override
+  public void flush() {
+    LOG.debug("ECKeyOutputStream does not support flush.");
+  }
+
+  private void closeCurrentStreamEntry()
+      throws IOException {
+    if (!blockOutputStreamEntryPool.isEmpty()) {
+      while (true) {
+        try {
+          BlockOutputStreamEntry entry =
+              blockOutputStreamEntryPool.getCurrentStreamEntry();
+          if (entry != null) {
+            try {
+              entry.close();
+            } catch (IOException ioe) {
+              handleException(entry, ioe);
+              continue;
+            }
+          }
+          return;
+        } catch (Exception e) {
+          markStreamClosed();
+          throw e;
+        }
+      }
+    }
+  }
+
+  /**
+   * Commit the key to OM, this will add the blocks as the new key blocks.
+   *
+   * @throws IOException
+   */
+  @Override
+  public void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    closed = true;
+    try {
+      final long lastStripeSize = getCurrentDataStripeSize();
+      if (isPartialStripe(lastStripeSize)) {
+        ByteBuffer bytesToWrite =
+            ecChunkBufferCache.getDataBuffers()[blockOutputStreamEntryPool
+                .getCurrentStreamEntry().getCurrentStreamIdx()];
+
+        // Finish writing the current partial cached chunk
+        if (bytesToWrite.position() % ecChunkSize != 0) {
+          final ECBlockOutputStreamEntry current =
+              blockOutputStreamEntryPool.getCurrentStreamEntry();
+          try {
+            byte[] array = bytesToWrite.array();
+            writeToOutputStream(current, array,
+                bytesToWrite.position(), 0, false);
+          } catch (Exception e) {
+            markStreamAsFailed(e);
+          }
+        }
+
+        final int parityCellSize =
+            (int) (lastStripeSize < ecChunkSize ? lastStripeSize : ecChunkSize);
+        addPadding(parityCellSize);
+        if (handleParityWrites(parityCellSize, true)
+            == StripeWriteStatus.FAILED) {
+          handleStripeFailure(lastStripeSize, true);
+        } else {
+          blockOutputStreamEntryPool.getCurrentStreamEntry()
+              .updateBlockGroupToAckedPosition(
+                  blockOutputStreamEntryPool.getCurrentStreamEntry()
+                      .getCurrentPosition());
+        }
+
+      }
+
+      closeCurrentStreamEntry();
+      Preconditions.checkArgument(writeOffset == offset,
+          "Expected writeOffset= " + writeOffset
+              + " Expected offset=" + offset);
+      blockOutputStreamEntryPool.commitKey(offset);
+    } finally {
+      blockOutputStreamEntryPool.cleanup();
+    }
+    ecChunkBufferCache.release();
+  }
+
+  private void handleStripeFailure(long lastStripeSize, boolean isClose)
+      throws IOException {
+    StripeWriteStatus stripeWriteStatus;
+    for (int i = 0; i < this.config.getMaxECStripeWriteRetries(); i++) {
+      stripeWriteStatus = rewriteStripeToNewBlockGroup(lastStripeSize, isClose);
+      if (stripeWriteStatus == StripeWriteStatus.SUCCESS) {
+        return;
+      }
+    }
+    throw new IOException("Completed max allowed retries " + this.config
+        .getMaxECStripeWriteRetries() + " on stripe failures.");
+
+  }
+
+  private void addPadding(int parityCellSize) {
+    ByteBuffer[] buffers = ecChunkBufferCache.getDataBuffers();
+
+    for (int i = 1; i < numDataBlks; i++) {
+      final int position = buffers[i].position();
+      assert position <= parityCellSize : "If an internal block is smaller"
+          + " than parity block, then its last cell should be small than last"
+          + " parity cell";
+      padBufferToLimit(buffers[i], parityCellSize);
+    }
+  }
+
+  public static void padBufferToLimit(ByteBuffer buf, int limit) {
+    int pos = buf.position();
+    if (pos >= limit) {
+      return;
+    }
+    Arrays.fill(buf.array(), pos, limit, (byte)0);
+    buf.position(limit);
+  }
+
+  private boolean isPartialStripe(long stripeSize) {
+    return stripeSize > 0 && stripeSize < (numDataBlks * ecChunkSize);
+  }
+
+  private long getCurrentDataStripeSize() {
+    final ByteBuffer[] dataBuffers = ecChunkBufferCache.getDataBuffers();
+    long lastStripeSize = 0;
+    for (int i = 0; i < numDataBlks; i++) {
+      lastStripeSize += dataBuffers[i].position();
+    }
+    return lastStripeSize;
+  }
+
+  public OmMultipartCommitUploadPartInfo getCommitUploadPartInfo() {
+    return blockOutputStreamEntryPool.getCommitUploadPartInfo();
+  }
+
+  @VisibleForTesting
+  public ExcludeList getExcludeList() {
+    return blockOutputStreamEntryPool.getExcludeList();
+  }
+
+  /**
+   * Builder class of ECKeyOutputStream.
+   */
+  public static class Builder extends KeyOutputStream.Builder {
+    private ECReplicationConfig replicationConfig;
+    private ByteBufferPool byteBufferPool;
+
+    @Override
+    public ECReplicationConfig getReplicationConfig() {
+      return replicationConfig;
+    }
+
+    public ECKeyOutputStream.Builder setReplicationConfig(
+        ECReplicationConfig replConfig) {
+      this.replicationConfig = replConfig;
+      return this;
+    }
+
+    public ByteBufferPool getByteBufferPool() {
+      return byteBufferPool;
+    }
+
+    public ECKeyOutputStream.Builder setByteBufferPool(
+        ByteBufferPool bufferPool) {
+      this.byteBufferPool = bufferPool;
+      return this;
+    }
+
+    @Override
+    public ECKeyOutputStream build() {
+      return new ECKeyOutputStream(this);
+    }
+  }
+
+  /**
+   * Verify that the output stream is open. Non blocking; this gives
+   * the last state of the volatile {@link #closed} field.
+   *
+   * @throws IOException if the connection is closed.
+   */
+  private void checkNotClosed() throws IOException {
+    if (closed) {
+      throw new IOException(
+          ": " + FSExceptionMessages.STREAM_IS_CLOSED + " Key: "
+              + blockOutputStreamEntryPool.getKeyName());
+    }
+  }
+
+  private static class ECChunkBuffers {
+    private final ByteBuffer[] dataBuffers;
+    private final ByteBuffer[] parityBuffers;
+    private int cellSize;
+    private ByteBufferPool byteBufferPool;
+
+    ECChunkBuffers(int cellSize, int numData, int numParity,
+        ByteBufferPool byteBufferPool) {
+      this.cellSize = cellSize;
+      dataBuffers = new ByteBuffer[numData];
+      parityBuffers = new ByteBuffer[numParity];
+      this.byteBufferPool = byteBufferPool;
+      allocateBuffers(dataBuffers, this.cellSize);
+      allocateBuffers(parityBuffers, this.cellSize);
+    }
+
+    private ByteBuffer[] getDataBuffers() {
+      return dataBuffers;
+    }
+
+    private ByteBuffer[] getParityBuffers() {
+      return parityBuffers;
+    }
+
+    private int addToDataBuffer(int i, byte[] b, int off, int len) {
+      final ByteBuffer buf = dataBuffers[i];
+      final int pos = buf.position() + len;
+      Preconditions.checkState(pos <= cellSize,
+          "Position(" + pos + ") is greater than the cellSize("
+              + cellSize + ").");
+      buf.put(b, off, len);
+      return pos;
+    }
+
+    private void clear() {
+      clearBuffers(dataBuffers);
+      clearBuffers(parityBuffers);
+    }
+
+    private void release() {
+      releaseBuffers(dataBuffers);
+      releaseBuffers(parityBuffers);
+    }
+
+    private void allocateBuffers(ByteBuffer[] buffers, int bufferSize) {
+      for (int i = 0; i < buffers.length; i++) {
+        buffers[i] = byteBufferPool.getBuffer(false, cellSize);
+        buffers[i].limit(bufferSize);
+      }
+    }
+
+    private void clearBuffers(ByteBuffer[] buffers) {
+      for (int i = 0; i < buffers.length; i++) {
+        buffers[i].clear();
+        buffers[i].limit(cellSize);
+      }
+    }
+
+    private void releaseBuffers(ByteBuffer[] buffers) {
+      for (int i = 0; i < buffers.length; i++) {
+        if (buffers[i] != null) {
+          byteBufferPool.putBuffer(buffers[i]);
+          buffers[i] = null;
+        }
+      }
+    }
+  }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
new file mode 100644
index 0000000..956ed90
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
@@ -0,0 +1,43 @@
+/**
+ * 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.ozone.client.io;
+
+import java.io.IOException;
+
+/**
+ * Exception thrown by EC Input Streams if there are not enough locations to
+ * read the EC data successfully.
+ */
+public class InsufficientLocationsException extends IOException {
+
+  public InsufficientLocationsException() {
+    super();
+  }
+
+  public InsufficientLocationsException(String message) {
+    super(message);
+  }
+
+  public InsufficientLocationsException(String message, Throwable ex) {
+    super(message, ex);
+  }
+
+  public InsufficientLocationsException(Throwable ex) {
+    super(ex);
+  }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
index f440cf9..20cb2b5 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
@@ -19,7 +19,6 @@
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.io.InputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -30,17 +29,17 @@
 import java.util.stream.Collectors;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.ByteBufferReadable;
-import org.apache.hadoop.fs.CanUnbuffer;
 import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.Seekable;
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.scm.XceiverClientFactory;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
 import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
 import org.apache.hadoop.hdds.scm.storage.ByteArrayReader;
 import org.apache.hadoop.hdds.scm.storage.ByteBufferReader;
 import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.scm.storage.ExtendedInputStream;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 
@@ -52,8 +51,7 @@
 /**
  * Maintaining a list of BlockInputStream. Read based on offset.
  */
-public class KeyInputStream extends InputStream
-    implements Seekable, CanUnbuffer, ByteBufferReadable {
+public class KeyInputStream extends ExtendedInputStream {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(KeyInputStream.class);
@@ -65,7 +63,7 @@
   private boolean closed = false;
 
   // List of BlockInputStreams, one for each block in the key
-  private final List<BlockInputStream> blockStreams;
+  private final List<BlockExtendedInputStream> blockStreams;
 
   // blockOffsets[i] stores the index of the first data byte in
   // blockStream w.r.t the key data.
@@ -93,20 +91,23 @@
    */
   public static LengthInputStream getFromOmKeyInfo(OmKeyInfo keyInfo,
       XceiverClientFactory xceiverClientFactory,
-      boolean verifyChecksum,  Function<OmKeyInfo, OmKeyInfo> retryFunction) {
+      boolean verifyChecksum,  Function<OmKeyInfo, OmKeyInfo> retryFunction,
+      BlockInputStreamFactory blockStreamFactory) {
     List<OmKeyLocationInfo> keyLocationInfos = keyInfo
         .getLatestVersionLocations().getBlocksLatestVersionOnly();
 
     KeyInputStream keyInputStream = new KeyInputStream();
     keyInputStream.initialize(keyInfo, keyLocationInfos,
-        xceiverClientFactory, verifyChecksum, retryFunction);
+        xceiverClientFactory, verifyChecksum, retryFunction,
+        blockStreamFactory);
 
     return new LengthInputStream(keyInputStream, keyInputStream.length);
   }
 
   public static List<LengthInputStream> getStreamsFromKeyInfo(OmKeyInfo keyInfo,
       XceiverClientFactory xceiverClientFactory, boolean verifyChecksum,
-      Function<OmKeyInfo, OmKeyInfo> retryFunction) {
+      Function<OmKeyInfo, OmKeyInfo> retryFunction,
+      BlockInputStreamFactory blockStreamFactory) {
     List<OmKeyLocationInfo> keyLocationInfos = keyInfo
         .getLatestVersionLocations().getBlocksLatestVersionOnly();
 
@@ -137,7 +138,8 @@
         partsToBlocksMap.entrySet()) {
       KeyInputStream keyInputStream = new KeyInputStream();
       keyInputStream.initialize(keyInfo, entry.getValue(),
-          xceiverClientFactory, verifyChecksum, retryFunction);
+          xceiverClientFactory, verifyChecksum, retryFunction,
+          blockStreamFactory);
       lengthInputStreams.add(new LengthInputStream(keyInputStream,
           partsLengthMap.get(entry.getKey())));
     }
@@ -148,7 +150,8 @@
   private synchronized void initialize(OmKeyInfo keyInfo,
       List<OmKeyLocationInfo> blockInfos,
       XceiverClientFactory xceiverClientFactory,
-      boolean verifyChecksum,  Function<OmKeyInfo, OmKeyInfo> retryFunction) {
+      boolean verifyChecksum,  Function<OmKeyInfo, OmKeyInfo> retryFunction,
+      BlockInputStreamFactory blockStreamFactory) {
     this.key = keyInfo.getKeyName();
     this.blockOffsets = new long[blockInfos.size()];
     long keyLength = 0;
@@ -161,7 +164,8 @@
 
       // We also pass in functional reference which is used to refresh the
       // pipeline info for a given OM Key location info.
-      addStream(omKeyLocationInfo, xceiverClientFactory,
+      addStream(keyInfo.getReplicationConfig(), omKeyLocationInfo,
+          xceiverClientFactory,
           verifyChecksum, keyLocationInfo -> {
             OmKeyInfo newKeyInfo = retryFunction.apply(keyInfo);
             BlockID blockID = keyLocationInfo.getBlockID();
@@ -176,7 +180,7 @@
             } else {
               return null;
             }
-          });
+          }, blockStreamFactory);
 
       this.blockOffsets[i] = keyLength;
       keyLength += omKeyLocationInfo.getLength();
@@ -190,12 +194,13 @@
    * BlockInputStream is initialized when a read operation is performed on
    * the block for the first time.
    */
-  private synchronized void addStream(OmKeyLocationInfo blockInfo,
-      XceiverClientFactory xceiverClientFactory,
-      boolean verifyChecksum,
-      Function<OmKeyLocationInfo, Pipeline> refreshPipelineFunction) {
-    blockStreams.add(new BlockInputStream(blockInfo.getBlockID(),
-        blockInfo.getLength(), blockInfo.getPipeline(), blockInfo.getToken(),
+  private synchronized void addStream(ReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo,
+      XceiverClientFactory xceiverClientFactory, boolean verifyChecksum,
+      Function<OmKeyLocationInfo, Pipeline> refreshPipelineFunction,
+      BlockInputStreamFactory blockStreamFactory) {
+    blockStreams.add(blockStreamFactory.create(repConfig, blockInfo,
+        blockInfo.getPipeline(), blockInfo.getToken(),
         verifyChecksum, xceiverClientFactory,
         blockID -> refreshPipelineFunction.apply(blockInfo)));
   }
@@ -205,6 +210,11 @@
     blockStreams.add(blockInputStream);
   }
 
+  @VisibleForTesting
+  public void addStream(BlockExtendedInputStream blockInputStream) {
+    blockStreams.add(blockInputStream);
+  }
+
   /**
    * {@inheritDoc}
    */
@@ -240,8 +250,9 @@
     return readWithStrategy(strategy);
   }
 
-  synchronized int readWithStrategy(ByteReaderStrategy strategy) throws
-      IOException {
+  @Override
+  protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+      throws IOException {
     Preconditions.checkArgument(strategy != null);
     checkOpen();
 
@@ -257,8 +268,8 @@
       }
 
       // Get the current blockStream and read data from it
-      BlockInputStream current = blockStreams.get(blockIndex);
-      int numBytesToRead = Math.min(buffLen, (int)current.getRemaining());
+      BlockExtendedInputStream current = blockStreams.get(blockIndex);
+      int numBytesToRead = (int)Math.min(buffLen, current.getRemaining());
       int numBytesRead = strategy.readFromBlock(current, numBytesToRead);
       if (numBytesRead != numBytesToRead) {
         // This implies that there is either data loss or corruption in the
@@ -326,7 +337,7 @@
     }
 
     // Reset the previous blockStream's position
-    blockStreams.get(blockIndexOfPrevPosition).resetPosition();
+    blockStreams.get(blockIndexOfPrevPosition).seek(0);
 
     // Reset all the blockStreams above the blockIndex. We do this to reset
     // any previous reads which might have updated the blockPosition and
@@ -360,7 +371,7 @@
   @Override
   public synchronized void close() throws IOException {
     closed = true;
-    for (BlockInputStream blockStream : blockStreams) {
+    for (ExtendedInputStream blockStream : blockStreams) {
       blockStream.close();
     }
   }
@@ -400,13 +411,13 @@
 
   @Override
   public synchronized void unbuffer() {
-    for (BlockInputStream is : blockStreams) {
+    for (ExtendedInputStream is : blockStreams) {
       is.unbuffer();
     }
   }
 
   @VisibleForTesting
-  public List<BlockInputStream> getBlockStreams() {
+  public List<BlockExtendedInputStream> getBlockStreams() {
     return blockStreams;
   }
 }
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java
index d5f6f5d..ee69ca1 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java
@@ -150,7 +150,6 @@
             unsafeByteBufferConversion,
             xceiverClientManager,
             handler.getId());
-
     this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException(
         config.getMaxRetryCount(), config.getRetryInterval());
     this.retryCount = 0;
@@ -432,7 +431,7 @@
 
   // Every container specific exception from datatnode will be seen as
   // StorageContainerException
-  private boolean checkIfContainerToExclude(Throwable t) {
+  protected boolean checkIfContainerToExclude(Throwable t) {
     return t instanceof StorageContainerException;
   }
 
@@ -554,56 +553,99 @@
     private OzoneClientConfig clientConfig;
     private ReplicationConfig replicationConfig;
 
+    public String getMultipartUploadID() {
+      return multipartUploadID;
+    }
+
     public Builder setMultipartUploadID(String uploadID) {
       this.multipartUploadID = uploadID;
       return this;
     }
 
+    public int getMultipartNumber() {
+      return multipartNumber;
+    }
+
     public Builder setMultipartNumber(int partNumber) {
       this.multipartNumber = partNumber;
       return this;
     }
 
+    public OpenKeySession getOpenHandler() {
+      return openHandler;
+    }
+
     public Builder setHandler(OpenKeySession handler) {
       this.openHandler = handler;
       return this;
     }
 
+    public XceiverClientFactory getXceiverManager() {
+      return xceiverManager;
+    }
+
     public Builder setXceiverClientManager(XceiverClientFactory manager) {
       this.xceiverManager = manager;
       return this;
     }
 
+    public OzoneManagerProtocol getOmClient() {
+      return omClient;
+    }
+
     public Builder setOmClient(OzoneManagerProtocol client) {
       this.omClient = client;
       return this;
     }
 
+    public int getChunkSize() {
+      return chunkSize;
+    }
+
     public Builder setChunkSize(int size) {
       this.chunkSize = size;
       return this;
     }
 
+    public String getRequestID() {
+      return requestID;
+    }
+
     public Builder setRequestID(String id) {
       this.requestID = id;
       return this;
     }
 
+    public boolean isMultipartKey() {
+      return isMultipartKey;
+    }
+
     public Builder setIsMultipartKey(boolean isMultipart) {
       this.isMultipartKey = isMultipart;
       return this;
     }
 
+    public OzoneClientConfig getClientConfig() {
+      return clientConfig;
+    }
+
     public Builder setConfig(OzoneClientConfig config) {
       this.clientConfig = config;
       return this;
     }
 
+    public boolean isUnsafeByteBufferConversionEnabled() {
+      return unsafeByteBufferConversion;
+    }
+
     public Builder enableUnsafeByteBufferConversion(boolean enabled) {
       this.unsafeByteBufferConversion = enabled;
       return this;
     }
 
+    public ReplicationConfig getReplicationConfig() {
+      return replicationConfig;
+    }
 
     public Builder setReplicationConfig(ReplicationConfig replConfig) {
       this.replicationConfig = replConfig;
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
index e2a8679..725eb3a 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
@@ -743,6 +743,17 @@
       long quotaInNamespace, long quotaInBytes) throws IOException;
 
   /**
+   * Set Bucket replication configuration.
+   *
+   * @param volumeName        Name of the Volume.
+   * @param bucketName        Name of the Bucket.
+   * @param replicationConfig The replication config to set on bucket.
+   * @throws IOException
+   */
+  void setReplicationConfig(String volumeName, String bucketName,
+      ReplicationConfig replicationConfig) throws IOException;
+
+  /**
    * Returns OzoneKey that contains the application generated/visible
    * metadata for an Ozone Object.
    *
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index 3a460de..213868c 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -37,14 +37,20 @@
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.crypto.CryptoInputStream;
 import org.apache.hadoop.crypto.CryptoOutputStream;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
@@ -62,6 +68,8 @@
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -78,6 +86,9 @@
 import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
 import org.apache.hadoop.ozone.client.io.KeyInputStream;
 import org.apache.hadoop.ozone.client.io.KeyOutputStream;
 import org.apache.hadoop.ozone.client.io.LengthInputStream;
@@ -156,6 +167,11 @@
   private static final Logger LOG =
       LoggerFactory.getLogger(RpcClient.class);
 
+  // For the minimal recommended EC policy rs-3-2-1024k,
+  // we should have at least 1 core thread for each necessary chunk
+  // for reconstruction.
+  private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;
+
   private final ConfigurationSource conf;
   private final OzoneManagerClientProtocol ozoneManagerClient;
   private final XceiverClientFactory xceiverClientManager;
@@ -172,6 +188,10 @@
   private final OzoneClientConfig clientConfig;
   private final Cache<URI, KeyProvider> keyProviderCache;
   private final boolean getLatestVersionLocation;
+  private final ByteBufferPool byteBufferPool;
+  private final BlockInputStreamFactory blockInputStreamFactory;
+  private final OzoneManagerVersion omVersion;
+  private volatile ExecutorService ecReconstructExecutor;
 
   /**
    * Creates RpcClient instance with the given configuration.
@@ -202,8 +222,9 @@
         OzoneManagerClientProtocol.class, conf);
     dtService = omTransport.getDelegationTokenService();
     List<X509Certificate> x509Certificates = null;
+    ServiceInfoEx serviceInfoEx = ozoneManagerClient.getServiceInfo();
+    omVersion = getOmVersion(serviceInfoEx);
     if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
-      ServiceInfoEx serviceInfoEx = ozoneManagerClient.getServiceInfo();
       // If the client is authenticating using S3 style auth, all future
       // requests serviced by this client will need S3 Auth set.
       boolean isS3 = conf.getBoolean(S3Auth.S3_AUTH_CHECK, false);
@@ -288,12 +309,30 @@
             }
           }
         }).build();
+    this.byteBufferPool = new ElasticByteBufferPool();
+    this.blockInputStreamFactory = BlockInputStreamFactoryImpl
+        .getInstance(byteBufferPool, this::getECReconstructExecutor);
   }
 
   public XceiverClientFactory getXceiverClientManager() {
     return xceiverClientManager;
   }
 
+  private OzoneManagerVersion getOmVersion(ServiceInfoEx info) {
+    OzoneManagerVersion version = OzoneManagerVersion.CURRENT;
+    for (ServiceInfo si : info.getServiceInfoList()) {
+      if (si.getNodeType() == HddsProtos.NodeType.OM) {
+        OzoneManagerVersion current =
+            OzoneManagerVersion.fromProtoValue(si.getProtobuf().getOMVersion());
+        if (version.compareTo(current) > 0) {
+          version = current;
+        }
+      }
+    }
+    LOG.trace("Ozone Manager version is {}", version.name());
+    return version;
+  }
+
   static boolean validateOmVersion(OzoneManagerVersion minimumVersion,
                                    List<ServiceInfo> serviceInfoList) {
     if (minimumVersion == OzoneManagerVersion.FUTURE_VERSION) {
@@ -532,6 +571,16 @@
     Preconditions.checkNotNull(bucketArgs);
     verifyCountsQuota(bucketArgs.getQuotaInNamespace());
     verifySpaceQuota(bucketArgs.getQuotaInBytes());
+    if (omVersion
+        .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+      if (bucketArgs.getDefaultReplicationConfig() != null &&
+          bucketArgs.getDefaultReplicationConfig().getType()
+          == ReplicationType.EC) {
+        throw new IOException("Can not set the default replication of the"
+            + " bucket to Erasure Coded replication, as OzoneManager does"
+            + " not support Erasure Coded replication.");
+      }
+    }
 
     final String owner;
     // If S3 auth exists, set owner name to the short user name derived from the
@@ -579,6 +628,12 @@
       builder.setBucketEncryptionKey(bek);
     }
 
+    DefaultReplicationConfig defaultReplicationConfig =
+        bucketArgs.getDefaultReplicationConfig();
+    if (defaultReplicationConfig != null) {
+      builder.setDefaultReplicationConfig(defaultReplicationConfig);
+    }
+
     LOG.info("Creating Bucket: {}/{}, with {} as owner and Versioning {} and " +
         "Storage Type set to {} and Encryption set to {} ",
         volumeName, bucketName, owner, isVersionEnabled,
@@ -769,6 +824,30 @@
   }
 
   @Override
+  public void setReplicationConfig(
+      String volumeName, String bucketName, ReplicationConfig replicationConfig)
+      throws IOException {
+    verifyVolumeName(volumeName);
+    verifyBucketName(bucketName);
+    Preconditions.checkNotNull(replicationConfig);
+    if (omVersion
+        .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+      if (replicationConfig.getReplicationType()
+          == HddsProtos.ReplicationType.EC) {
+        throw new IOException("Can not set the default replication of the"
+            + " bucket to Erasure Coded replication, as OzoneManager does"
+            + " not support Erasure Coded replication.");
+      }
+    }
+    OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setDefaultReplicationConfig(
+            new DefaultReplicationConfig(replicationConfig));
+    ozoneManagerClient.setBucketProperty(builder.build());
+  }
+
+  @Override
   public void deleteBucket(
       String volumeName, String bucketName) throws IOException {
     verifyVolumeName(volumeName);
@@ -808,7 +887,8 @@
         bucketInfo.getQuotaInBytes(),
         bucketInfo.getQuotaInNamespace(),
         bucketInfo.getBucketLayout(),
-        bucketInfo.getOwner()
+        bucketInfo.getOwner(),
+        bucketInfo.getDefaultReplicationConfig()
     );
   }
 
@@ -838,7 +918,8 @@
         bucket.getQuotaInBytes(),
         bucket.getQuotaInNamespace(),
         bucket.getBucketLayout(),
-        bucket.getOwner()))
+        bucket.getOwner(),
+        bucket.getDefaultReplicationConfig()))
         .collect(Collectors.toList());
   }
 
@@ -863,7 +944,17 @@
     if (checkKeyNameEnabled) {
       HddsClientUtils.verifyKeyName(keyName);
     }
-    HddsClientUtils.checkNotNull(keyName, replicationConfig);
+    HddsClientUtils.checkNotNull(keyName);
+    if (omVersion
+        .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+      if (replicationConfig != null &&
+          replicationConfig.getReplicationType()
+              == HddsProtos.ReplicationType.EC) {
+        throw new IOException("Can not set the replication of the key to"
+            + " Erasure Coded replication, as OzoneManager does not support"
+            + " Erasure Coded replication.");
+      }
+    }
     String requestId = UUID.randomUUID().toString();
 
     OmKeyArgs.Builder builder = new OmKeyArgs.Builder()
@@ -891,7 +982,7 @@
     }
 
     OpenKeySession openKey = ozoneManagerClient.openKey(builder.build());
-    return createOutputStream(openKey, requestId, replicationConfig);
+    return createOutputStream(openKey, requestId);
   }
 
   private KeyProvider.KeyVersion getDEK(FileEncryptionInfo feInfo)
@@ -1133,6 +1224,10 @@
 
   @Override
   public void close() throws IOException {
+    if (ecReconstructExecutor != null) {
+      ecReconstructExecutor.shutdownNow();
+      ecReconstructExecutor = null;
+    }
     IOUtils.cleanupWithLogger(LOG, ozoneManagerClient, xceiverClientManager);
     keyProviderCache.invalidateAll();
     keyProviderCache.cleanUp();
@@ -1155,7 +1250,16 @@
       throws IOException {
     verifyVolumeName(volumeName);
     verifyBucketName(bucketName);
-    HddsClientUtils.checkNotNull(keyName, replicationConfig);
+    HddsClientUtils.checkNotNull(keyName);
+    if (omVersion
+        .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+      if (replicationConfig.getReplicationType()
+          == HddsProtos.ReplicationType.EC) {
+        throw new IOException("Can not set the replication of the file to"
+            + " Erasure Coded replication, as OzoneManager does not support"
+            + " Erasure Coded replication.");
+      }
+    }
     OmKeyArgs keyArgs = new OmKeyArgs.Builder()
         .setVolumeName(volumeName)
         .setBucketName(bucketName)
@@ -1199,19 +1303,11 @@
         .build();
 
     OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs);
-    KeyOutputStream keyOutputStream =
-        new KeyOutputStream.Builder()
-            .setHandler(openKey)
-            .setXceiverClientManager(xceiverClientManager)
-            .setOmClient(ozoneManagerClient)
-            .setRequestID(requestId)
-            .setReplicationConfig(openKey.getKeyInfo().getReplicationConfig())
-            .setMultipartNumber(partNumber)
-            .setMultipartUploadID(uploadID)
-            .setIsMultipartKey(true)
-            .enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
-            .setConfig(clientConfig)
-            .build();
+    KeyOutputStream keyOutputStream = createKeyOutputStream(openKey, requestId)
+        .setMultipartNumber(partNumber)
+        .setMultipartUploadID(uploadID)
+        .setIsMultipartKey(true)
+        .build();
     keyOutputStream.addPreallocateBlocks(
         openKey.getKeyInfo().getLatestVersionLocations(),
         openKey.getOpenVersion());
@@ -1403,6 +1499,15 @@
   public OzoneOutputStream createFile(String volumeName, String bucketName,
       String keyName, long size, ReplicationConfig replicationConfig,
       boolean overWrite, boolean recursive) throws IOException {
+    if (omVersion
+        .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+      if (replicationConfig.getReplicationType()
+          == HddsProtos.ReplicationType.EC) {
+        throw new IOException("Can not set the replication of the file to"
+            + " Erasure Coded replication, as OzoneManager does not support"
+            + " Erasure Coded replication.");
+      }
+    }
     OmKeyArgs keyArgs = new OmKeyArgs.Builder()
         .setVolumeName(volumeName)
         .setBucketName(bucketName)
@@ -1414,8 +1519,7 @@
         .build();
     OpenKeySession keySession =
         ozoneManagerClient.createFile(keyArgs, overWrite, recursive);
-    return createOutputStream(keySession, UUID.randomUUID().toString(),
-        replicationConfig);
+    return createOutputStream(keySession, UUID.randomUUID().toString());
   }
 
   @Override
@@ -1494,7 +1598,8 @@
     if (feInfo == null) {
       LengthInputStream lengthInputStream = KeyInputStream
           .getFromOmKeyInfo(keyInfo, xceiverClientManager,
-              clientConfig.isChecksumVerify(), retryFunction);
+              clientConfig.isChecksumVerify(), retryFunction,
+              blockInputStreamFactory);
       try {
         Map< String, String > keyInfoMetadata = keyInfo.getMetadata();
         if (Boolean.valueOf(keyInfoMetadata.get(OzoneConsts.GDPR_FLAG))) {
@@ -1514,7 +1619,8 @@
       // Regular Key with FileEncryptionInfo
       LengthInputStream lengthInputStream = KeyInputStream
           .getFromOmKeyInfo(keyInfo, xceiverClientManager,
-              clientConfig.isChecksumVerify(), retryFunction);
+              clientConfig.isChecksumVerify(), retryFunction,
+              blockInputStreamFactory);
       final KeyProvider.KeyVersion decrypted = getDEK(feInfo);
       final CryptoInputStream cryptoIn =
           new CryptoInputStream(lengthInputStream.getWrappedStream(),
@@ -1525,7 +1631,8 @@
       // Multipart Key with FileEncryptionInfo
       List<LengthInputStream> lengthInputStreams = KeyInputStream
           .getStreamsFromKeyInfo(keyInfo, xceiverClientManager,
-              clientConfig.isChecksumVerify(), retryFunction);
+              clientConfig.isChecksumVerify(), retryFunction,
+              blockInputStreamFactory);
       final KeyProvider.KeyVersion decrypted = getDEK(feInfo);
 
       List<OzoneCryptoInputStream> cryptoInputStreams = new ArrayList<>();
@@ -1542,18 +1649,11 @@
   }
 
   private OzoneOutputStream createOutputStream(OpenKeySession openKey,
-      String requestId, ReplicationConfig replicationConfig)
-      throws IOException {
-    KeyOutputStream keyOutputStream =
-        new KeyOutputStream.Builder()
-            .setHandler(openKey)
-            .setXceiverClientManager(xceiverClientManager)
-            .setOmClient(ozoneManagerClient)
-            .setRequestID(requestId)
-            .setReplicationConfig(replicationConfig)
-            .enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
-            .setConfig(clientConfig)
-            .build();
+      String requestId) throws IOException {
+
+    KeyOutputStream keyOutputStream = createKeyOutputStream(openKey, requestId)
+        .build();
+
     keyOutputStream
         .addPreallocateBlocks(openKey.getKeyInfo().getLatestVersionLocations(),
             openKey.getOpenVersion());
@@ -1588,6 +1688,30 @@
     }
   }
 
+  private KeyOutputStream.Builder createKeyOutputStream(OpenKeySession openKey,
+      String requestId) {
+    KeyOutputStream.Builder builder;
+
+    ReplicationConfig replicationConfig =
+        openKey.getKeyInfo().getReplicationConfig();
+    if (replicationConfig.getReplicationType() ==
+        HddsProtos.ReplicationType.EC) {
+      builder = new ECKeyOutputStream.Builder()
+          .setReplicationConfig((ECReplicationConfig) replicationConfig)
+          .setByteBufferPool(byteBufferPool);
+    } else {
+      builder = new KeyOutputStream.Builder()
+        .setReplicationConfig(replicationConfig);
+    }
+
+    return builder.setHandler(openKey)
+        .setXceiverClientManager(xceiverClientManager)
+        .setOmClient(ozoneManagerClient)
+        .setRequestID(requestId)
+        .enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
+        .setConfig(clientConfig);
+  }
+
   @Override
   public KeyProvider getKeyProvider() throws IOException {
     URI kmsUri = getKeyProviderUri();
@@ -1680,4 +1804,27 @@
         .setOwnerName(owner);
     return ozoneManagerClient.setBucketOwner(builder.build());
   }
+
+  public ExecutorService getECReconstructExecutor() {
+    // local ref to a volatile to ensure access
+    // to a completed initialized object
+    ExecutorService executor = ecReconstructExecutor;
+    if (executor == null) {
+      synchronized (this) {
+        executor = ecReconstructExecutor;
+        if (executor == null) {
+          ecReconstructExecutor = new ThreadPoolExecutor(
+              EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
+              clientConfig.getEcReconstructStripeReadPoolLimit(),
+              60, TimeUnit.SECONDS, new SynchronousQueue<>(),
+              new ThreadFactoryBuilder()
+                  .setNameFormat("ec-reconstruct-reader-TID-%d")
+                  .build(),
+              new ThreadPoolExecutor.CallerRunsPolicy());
+          executor = ecReconstructExecutor;
+        }
+      }
+    }
+    return executor;
+  }
 }
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockBlockAllocator.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockBlockAllocator.java
index 747b39b..0d5e1a2 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockBlockAllocator.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockBlockAllocator.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.client;
 
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation;
 
@@ -27,7 +28,7 @@
  */
 public interface MockBlockAllocator {
 
-  Iterable<? extends KeyLocation> allocateBlock(
-      KeyArgs createKeyRequest);
+  Iterable<? extends KeyLocation> allocateBlock(KeyArgs createKeyRequest,
+      ExcludeList excludeList);
 
 }
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java
index cb9875b..1f29425 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java
@@ -18,11 +18,13 @@
 package org.apache.hadoop.ozone.client;
 
 import org.apache.commons.collections.map.HashedMap;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -32,11 +34,18 @@
 public class MockDatanodeStorage {
 
   private final Map<DatanodeBlockID, BlockData> blocks = new HashedMap();
+  private final Map<BlockID, String> fullBlockData = new HashMap<>();
 
   private final Map<String, ChunkInfo> chunks = new HashMap<>();
 
   private final Map<String, ByteString> data = new HashMap<>();
 
+  private boolean failed = false;
+
+  public void setStorageFailed() {
+    this.failed = true;
+  }
+
   public void putBlock(DatanodeBlockID blockID, BlockData blockData) {
     blocks.put(blockID, blockData);
   }
@@ -47,9 +56,17 @@
 
   public void writeChunk(
       DatanodeBlockID blockID,
-      ChunkInfo chunkInfo, ByteString bytes) {
-    data.put(createKey(blockID, chunkInfo), bytes);
+      ChunkInfo chunkInfo, ByteString bytes) throws IOException {
+    if (failed) {
+      throw new IOException("This storage was marked as failed.");
+    }
+    data.put(createKey(blockID, chunkInfo),
+        ByteString.copyFrom(bytes.toByteArray()));
     chunks.put(createKey(blockID, chunkInfo), chunkInfo);
+    fullBlockData
+        .put(new BlockID(blockID.getContainerID(), blockID.getLocalID()),
+            fullBlockData.getOrDefault(blockID, "")
+                .concat(bytes.toStringUtf8()));
   }
 
   public ChunkInfo readChunkInfo(
@@ -70,4 +87,12 @@
         + chunkInfo.getChunkName() + "_" + chunkInfo.getOffset();
   }
 
+  public Map<String, ByteString> getAllBlockData() {
+    return this.data;
+  }
+
+  public String getFullBlockData(BlockID blockID) {
+    return this.fullBlockData.get(blockID);
+  }
+
 }
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java
index 17a7f6b..90ed563 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java
@@ -17,6 +17,14 @@
  */
 package org.apache.hadoop.ozone.client;
 
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ozone.om.protocolPB.OmTransport;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
@@ -50,6 +58,7 @@
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.function.Function;
 
@@ -75,7 +84,7 @@
   }
 
   public MockOmTransport() {
-    this.blockAllocator = new SinglePipelineBlockAllocator();
+    this(new SinglePipelineBlockAllocator(new OzoneConfiguration()));
   }
 
   @Override
@@ -129,10 +138,18 @@
 
   private OzoneManagerProtocolProtos.AllocateBlockResponse allocateBlock(
       OzoneManagerProtocolProtos.AllocateBlockRequest allocateBlockRequest) {
-    return OzoneManagerProtocolProtos.AllocateBlockResponse.newBuilder()
-        .setKeyLocation(
-            blockAllocator.allocateBlock(allocateBlockRequest.getKeyArgs())
-                .iterator().next()).build();
+    Iterator<? extends OzoneManagerProtocolProtos.KeyLocation> iterator =
+        blockAllocator.allocateBlock(allocateBlockRequest.getKeyArgs(),
+            ExcludeList.getFromProtoBuf(allocateBlockRequest.getExcludeList()))
+            .iterator();
+    OzoneManagerProtocolProtos.AllocateBlockResponse.Builder builder =
+        OzoneManagerProtocolProtos.AllocateBlockResponse.newBuilder()
+            .setKeyLocation(iterator.next());
+    while (iterator.hasNext()) {
+      builder.mergeKeyLocation(iterator.next());
+    }
+    return builder.build();
+
   }
 
   private DeleteVolumeResponse deleteVolume(
@@ -153,11 +170,28 @@
 
   private CommitKeyResponse commitKey(CommitKeyRequest commitKeyRequest) {
     final KeyArgs keyArgs = commitKeyRequest.getKeyArgs();
-    final KeyInfo remove =
+    final KeyInfo openKey =
         openKeys.get(keyArgs.getVolumeName()).get(keyArgs.getBucketName())
             .remove(keyArgs.getKeyName());
+    final KeyInfo.Builder committedKeyInfoWithLocations =
+        KeyInfo.newBuilder().setVolumeName(keyArgs.getVolumeName())
+            .setBucketName(keyArgs.getBucketName())
+            .setKeyName(keyArgs.getKeyName())
+            .setCreationTime(openKey.getCreationTime())
+            .setModificationTime(openKey.getModificationTime())
+            .setDataSize(keyArgs.getDataSize()).setLatestVersion(0L)
+            .addKeyLocationList(KeyLocationList.newBuilder()
+                .addAllKeyLocations(keyArgs.getKeyLocationsList()));
+    // Just inherit replication config details from open Key
+    if (openKey.hasEcReplicationConfig()) {
+      committedKeyInfoWithLocations
+          .setEcReplicationConfig(openKey.getEcReplicationConfig());
+    } else if (openKey.hasFactor()) {
+      committedKeyInfoWithLocations.setFactor(openKey.getFactor());
+    }
+    committedKeyInfoWithLocations.setType(openKey.getType());
     keys.get(keyArgs.getVolumeName()).get(keyArgs.getBucketName())
-        .put(keyArgs.getKeyName(), remove);
+        .put(keyArgs.getKeyName(), committedKeyInfoWithLocations.build());
     return CommitKeyResponse.newBuilder()
         .build();
   }
@@ -165,33 +199,95 @@
   private CreateKeyResponse createKey(CreateKeyRequest createKeyRequest) {
     final KeyArgs keyArgs = createKeyRequest.getKeyArgs();
     final long now = System.currentTimeMillis();
-    final KeyInfo keyInfo = KeyInfo.newBuilder()
-        .setVolumeName(keyArgs.getVolumeName())
-        .setBucketName(keyArgs.getBucketName())
-        .setKeyName(keyArgs.getKeyName())
-        .setCreationTime(now)
-        .setModificationTime(now)
-        .setType(keyArgs.getType())
-        .setFactor(keyArgs.getFactor())
-        .setDataSize(keyArgs.getDataSize())
-        .setLatestVersion(0L)
-        .addKeyLocationList(KeyLocationList.newBuilder()
-            .addAllKeyLocations(
-                blockAllocator.allocateBlock(createKeyRequest.getKeyArgs()))
-            .build())
-        .build();
+    final BucketInfo bucketInfo =
+        buckets.get(keyArgs.getVolumeName()).get(keyArgs.getBucketName());
+
+    final KeyInfo.Builder keyInfoBuilder =
+        KeyInfo.newBuilder().setVolumeName(keyArgs.getVolumeName())
+            .setBucketName(keyArgs.getBucketName())
+            .setKeyName(keyArgs.getKeyName()).setCreationTime(now)
+            .setModificationTime(now).setDataSize(keyArgs.getDataSize())
+            .setLatestVersion(0L).addKeyLocationList(
+            KeyLocationList.newBuilder().addAllKeyLocations(
+                blockAllocator.allocateBlock(createKeyRequest.getKeyArgs(),
+                    new ExcludeList()))
+                .build());
+
+    if (keyArgs.getType() == HddsProtos.ReplicationType.NONE) {
+      // 1. Client did not pass replication config.
+      // Now lets try bucket defaults
+      if (bucketInfo.getDefaultReplicationConfig() != null) {
+        // Since Bucket defaults are available, let's inherit
+        final HddsProtos.ReplicationType type =
+            bucketInfo.getDefaultReplicationConfig().getType();
+        keyInfoBuilder
+            .setType(bucketInfo.getDefaultReplicationConfig().getType());
+        switch (type) {
+        case EC:
+          keyInfoBuilder.setEcReplicationConfig(
+              bucketInfo.getDefaultReplicationConfig()
+                  .getEcReplicationConfig());
+          break;
+        case RATIS:
+        case STAND_ALONE:
+          keyInfoBuilder
+              .setFactor(bucketInfo.getDefaultReplicationConfig().getFactor());
+          break;
+        default:
+          throw new UnsupportedOperationException(
+              "Unknown replication type: " + type);
+        }
+      } else {
+        keyInfoBuilder.setType(HddsProtos.ReplicationType.RATIS);
+        keyInfoBuilder.setFactor(HddsProtos.ReplicationFactor.THREE);
+      }
+    } else {
+      // 1. Client passed the replication config.
+      // Let's use it.
+      final HddsProtos.ReplicationType type = keyArgs.getType();
+      keyInfoBuilder.setType(type);
+      switch (type) {
+      case EC:
+        keyInfoBuilder.setEcReplicationConfig(keyArgs.getEcReplicationConfig());
+        break;
+      case RATIS:
+      case STAND_ALONE:
+        keyInfoBuilder.setFactor(keyArgs.getFactor());
+        break;
+      default:
+        throw new UnsupportedOperationException(
+            "Unknown replication type: " + type);
+      }
+    }
+
+    final KeyInfo keyInfo = keyInfoBuilder.build();
     openKeys.get(keyInfo.getVolumeName()).get(keyInfo.getBucketName())
         .put(keyInfo.getKeyName(), keyInfo);
-    return CreateKeyResponse.newBuilder()
-        .setOpenVersion(0L)
-        .setKeyInfo(keyInfo)
+    return CreateKeyResponse.newBuilder().setOpenVersion(0L).setKeyInfo(keyInfo)
         .build();
   }
 
   private InfoBucketResponse infoBucket(InfoBucketRequest infoBucketRequest) {
+    BucketInfo bucketInfo = buckets.get(infoBucketRequest.getVolumeName())
+        .get(infoBucketRequest.getBucketName());
+    if (!bucketInfo.hasDefaultReplicationConfig()) {
+      final ReplicationConfig replicationConfig = ReplicationConfig
+          .getDefault(new OzoneConfiguration());
+
+      bucketInfo = bucketInfo.toBuilder().setDefaultReplicationConfig(
+          new DefaultReplicationConfig(
+              ReplicationType.fromProto(replicationConfig.getReplicationType()),
+              replicationConfig
+                  .getReplicationType() != HddsProtos.ReplicationType.EC ?
+                  ReplicationFactor
+                      .valueOf(replicationConfig.getRequiredNodes()) :
+                  null, replicationConfig
+              .getReplicationType() == HddsProtos.ReplicationType.EC ?
+              (ECReplicationConfig) replicationConfig :
+              null).toProto()).build();
+    }
     return InfoBucketResponse.newBuilder()
-        .setBucketInfo(buckets.get(infoBucketRequest.getVolumeName())
-            .get(infoBucketRequest.getBucketName()))
+        .setBucketInfo(bucketInfo)
         .build();
   }
 
@@ -258,6 +354,10 @@
     return CreateBucketResponse.newBuilder().build();
   }
 
+  public Map<String, Map<String, Map<String, KeyInfo>>> getKeys() {
+    return this.keys;
+  }
+
   @Override
   public Text getDelegationTokenService() {
     return null;
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java
index a5fa2bb..fbcc153 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java
@@ -23,7 +23,10 @@
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -34,6 +37,31 @@
 
   private final Map<DatanodeDetails, MockDatanodeStorage> storage =
       new HashMap<>();
+  private List<DatanodeDetails> pendingToFailNodes = new ArrayList<>();
+
+  public void setFailedStorages(List<DatanodeDetails> failedStorages) {
+    List<DatanodeDetails> remainingFailNodes = new ArrayList<>();
+    for (int i = 0; i < failedStorages.size(); i++) {
+      DatanodeDetails failedDN = failedStorages.get(i);
+      boolean isCurrentNodeMarked = false;
+      final Iterator<Map.Entry<DatanodeDetails, MockDatanodeStorage>> iterator =
+          storage.entrySet().iterator();
+      while (iterator.hasNext()) {
+        final Map.Entry<DatanodeDetails, MockDatanodeStorage> next =
+            iterator.next();
+        if (next.getKey().equals(failedDN)) {
+          final MockDatanodeStorage value = next.getValue();
+          value.setStorageFailed();
+          isCurrentNodeMarked = true;
+        }
+      }
+      if (!isCurrentNodeMarked) {
+        //This node does not initialized by client yet.
+        remainingFailNodes.add(failedDN);
+      }
+    }
+    this.pendingToFailNodes = remainingFailNodes;
+  }
 
   @Override
   public void close() throws IOException {
@@ -43,9 +71,13 @@
   @Override
   public XceiverClientSpi acquireClient(Pipeline pipeline)
       throws IOException {
-    return new MockXceiverClientSpi(pipeline, storage
-        .computeIfAbsent(pipeline.getFirstNode(),
-            r -> new MockDatanodeStorage()));
+    MockXceiverClientSpi mockXceiverClientSpi =
+        new MockXceiverClientSpi(pipeline, storage
+            .computeIfAbsent(pipeline.getFirstNode(),
+                r -> new MockDatanodeStorage()));
+    // Incase if this node already set to mark as failed.
+    setFailedStorages(this.pendingToFailNodes);
+    return mockXceiverClientSpi;
   }
 
   @Override
@@ -67,6 +99,13 @@
       boolean b) {
 
   }
-};
+
+  /**
+   * Returns data nodes details.
+   */
+  public Map<DatanodeDetails, MockDatanodeStorage> getStorages() {
+    return this.storage;
+  }
+}
 
 
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java
index 4d3db44..4dfe966 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java
@@ -38,6 +38,7 @@
 import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 
+import java.io.IOException;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
@@ -82,7 +83,13 @@
     switch (request.getCmdType()) {
     case WriteChunk:
       return result(request,
-          r -> r.setWriteChunk(writeChunk(request.getWriteChunk())));
+          r -> {
+            try {
+              return r.setWriteChunk(writeChunk(request.getWriteChunk()));
+            } catch (IOException e) {
+              return r.setResult(Result.IO_EXCEPTION);
+            }
+          });
     case ReadChunk:
       return result(request,
           r -> r.setReadChunk(readChunk(request.getReadChunk())));
@@ -149,7 +156,7 @@
   }
 
   private WriteChunkResponseProto writeChunk(
-      WriteChunkRequestProto writeChunk) {
+      WriteChunkRequestProto writeChunk) throws IOException {
     datanodeStorage
         .writeChunk(writeChunk.getBlockID(), writeChunk.getChunkData(),
             writeChunk.getData());
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MultiNodePipelineBlockAllocator.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MultiNodePipelineBlockAllocator.java
new file mode 100644
index 0000000..97c5cfd
--- /dev/null
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MultiNodePipelineBlockAllocator.java
@@ -0,0 +1,143 @@
+/*
+ * 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.ozone.client;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Allocates the block with required number of nodes in the pipeline.
+ * The nodes are pre-created with port numbers starting from 0 to
+ * ( given cluster size -1).
+ */
+public class MultiNodePipelineBlockAllocator implements MockBlockAllocator {
+  private long blockId;
+  private int requiredNodes;
+  private final ConfigurationSource conf;
+  private List<HddsProtos.DatanodeDetailsProto> clusterDns = new ArrayList<>();
+  private int start = 0;
+
+  public MultiNodePipelineBlockAllocator(OzoneConfiguration conf,
+      int requiredNodes, int clusterSize) {
+    this.requiredNodes = requiredNodes;
+    this.conf = conf;
+    // Pre-initializing the datanodes. Later allocateBlock API will use this
+    // nodes to add the required number of nodes in the block pipelines.
+    for (int i = 0; i < clusterSize; i++) {
+      clusterDns.add(HddsProtos.DatanodeDetailsProto.newBuilder().setUuid128(
+          HddsProtos.UUID.newBuilder().setLeastSigBits(i).setMostSigBits(i)
+              .build()).setHostName("localhost").setIpAddress("1.2.3.4")
+          .addPorts(HddsProtos.Port.newBuilder().setName("RATIS").setValue(i)
+              .build()).build());
+    }
+  }
+
+  public List<HddsProtos.DatanodeDetailsProto> getClusterDns() {
+    return this.clusterDns;
+  }
+
+  /**
+   * This method selects the block pipeline nodes from the pre-created cluster
+   * nodes(clusterDns). It will use requiredNodes field to decide how many nodes
+   * to be chosen for the pipeline. To make the tests easy prediction of the
+   * node allocations, it will choose block pipeline nodes in a sliding window
+   * fashion starting from 0th index in clusterDns in incrementing order until
+   * given requireNodes number. Similarly for the next block pipeline, it will
+   * start from the index location of previous chosen pipeline's last node index
+   * + 1. Let's say cluster size was initialized with 10 and required nodes are
+   * 5, the first block pipeline will have nodes from 0 to 4 and the second
+   * block will be assigned with the index locations of 5th to 9th nodes. Once
+   * we finish round of allocations, then it will start from 0 again for next
+   * block. It will also support exclude list. If client passes exclude list, it
+   * will simply skip the node if it presents in exclude list, instead it will
+   * simply take the next node. If not enough nodes left due to the grown
+   * exclude list, it will throw IllegalStateException.
+   *
+   * @param keyArgs
+   * @param excludeList
+   * @return KeyLocation
+   */
+  @Override
+  public Iterable<? extends OzoneManagerProtocolProtos.KeyLocation>
+      allocateBlock(OzoneManagerProtocolProtos.KeyArgs keyArgs,
+      ExcludeList excludeList) {
+    HddsProtos.Pipeline.Builder builder =
+        HddsProtos.Pipeline.newBuilder().setFactor(keyArgs.getFactor())
+            .setType(keyArgs.getType()).setId(HddsProtos.PipelineID.newBuilder()
+            .setUuid128(HddsProtos.UUID.newBuilder().setLeastSigBits(1L)
+                .setMostSigBits(1L).build()).build());
+    addMembers(builder, requiredNodes, excludeList.getDatanodes(), keyArgs);
+    if (keyArgs.getType() == HddsProtos.ReplicationType.EC) {
+      builder.setEcReplicationConfig(keyArgs.getEcReplicationConfig());
+    }
+    final HddsProtos.Pipeline pipeline = builder.build();
+    List<OzoneManagerProtocolProtos.KeyLocation> results = new ArrayList<>();
+    long blockSize = (long) conf
+        .getStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
+            OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
+    results.add(OzoneManagerProtocolProtos.KeyLocation.newBuilder()
+        .setPipeline(pipeline).setBlockID(
+            HddsProtos.BlockID.newBuilder().setBlockCommitSequenceId(1L)
+                .setContainerBlockID(
+                    HddsProtos.ContainerBlockID.newBuilder().setContainerID(1L)
+                        .setLocalID(blockId++).build()).build()).setOffset(0L)
+        .setLength(blockSize).build());
+    return results;
+  }
+
+  private void addMembers(HddsProtos.Pipeline.Builder builder, int nodesNeeded,
+      Set<DatanodeDetails> excludedDataNodes,
+      OzoneManagerProtocolProtos.KeyArgs keyArgs) {
+    int clusterSize = clusterDns.size();
+    int counter = nodesNeeded;
+    int j = 0;
+    for (int i = 0; i < clusterDns.size(); i++) {
+      HddsProtos.DatanodeDetailsProto datanodeDetailsProto =
+          clusterDns.get(start % clusterSize);
+      start++;
+      if (excludedDataNodes
+          .contains(DatanodeDetails.getFromProtoBuf(datanodeDetailsProto))) {
+        continue;
+      } else {
+        builder.addMembers(datanodeDetailsProto);
+        if (keyArgs.getType() == HddsProtos.ReplicationType.EC) {
+          builder.addMemberReplicaIndexes(++j);
+        }
+        if (--counter == 0) {
+          break;
+        }
+      }
+    }
+    if (counter > 0) {
+      throw new IllegalStateException(
+          "MockedImpl: Could not find enough nodes.");
+    }
+  }
+
+}
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/SinglePipelineBlockAllocator.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/SinglePipelineBlockAllocator.java
index 71de5e8..20fbc03 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/SinglePipelineBlockAllocator.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/SinglePipelineBlockAllocator.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.ozone.client;
 
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockID;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerBlockID;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
@@ -24,6 +27,8 @@
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.PipelineID;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.Port;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.UUID;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation;
 
@@ -38,17 +43,18 @@
 
   private long blockId;
   private Pipeline pipeline;
+  private OzoneConfiguration conf;
 
-  public SinglePipelineBlockAllocator() {
-
+  public SinglePipelineBlockAllocator(OzoneConfiguration conf) {
+    this.conf = conf;
   }
 
   @Override
-  public Iterable<? extends KeyLocation> allocateBlock(
-      KeyArgs keyArgs) {
+  public Iterable<? extends KeyLocation> allocateBlock(KeyArgs keyArgs,
+      ExcludeList excludeList) {
 
     if (pipeline == null) {
-      pipeline = Pipeline.newBuilder()
+      Pipeline.Builder bldr = Pipeline.newBuilder()
           .setFactor(keyArgs.getFactor())
           .setType(keyArgs.getType())
           .setId(PipelineID.newBuilder()
@@ -68,10 +74,18 @@
                   .setName("RATIS")
                   .setValue(1234)
                   .build())
-              .build())
-          .build();
+              .build());
+      if (keyArgs.getType() == HddsProtos.ReplicationType.EC) {
+        bldr.setEcReplicationConfig(keyArgs.getEcReplicationConfig());
+      }
+      pipeline = bldr.build();
     }
 
+    long blockSize =  (long)conf.getStorageSize(
+        OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
+        OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT,
+        StorageUnit.BYTES);
+
     List<KeyLocation> results = new ArrayList<>();
     results.add(KeyLocation.newBuilder()
         .setPipeline(pipeline)
@@ -83,7 +97,7 @@
                 .build())
             .build())
         .setOffset(0L)
-        .setLength(keyArgs.getDataSize())
+        .setLength(blockSize)
         .build());
     return results;
   }
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClient.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClient.java
index 2f4ebcf..23b181f 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClient.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClient.java
@@ -18,10 +18,13 @@
 
 package org.apache.hadoop.ozone.client;
 
+import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
-import org.apache.hadoop.hdds.conf.InMemoryConfiguration;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
@@ -70,21 +73,24 @@
 
   @Before
   public void init() throws IOException {
-    ConfigurationSource config = new InMemoryConfiguration();
+    OzoneConfiguration config = new OzoneConfiguration();
+    createNewClient(config, new SinglePipelineBlockAllocator(config));
+  }
+
+  private void createNewClient(ConfigurationSource config,
+      MockBlockAllocator blkAllocator) throws IOException {
     client = new OzoneClient(config, new RpcClient(config, null) {
 
       @Override
-      protected OmTransport createOmTransport(
-          String omServiceId)
+      protected OmTransport createOmTransport(String omServiceId)
           throws IOException {
-        return new MockOmTransport();
+        return new MockOmTransport(blkAllocator);
       }
 
       @NotNull
       @Override
       protected XceiverClientFactory createXceiverClientFactory(
-          List<X509Certificate> x509Certificates)
-          throws IOException {
+          List<X509Certificate> x509Certificates) throws IOException {
         return new MockXceiverClientFactory();
       }
     });
@@ -183,6 +189,35 @@
     }
   }
 
+  @Test
+  public void testPutKeyWithECReplicationConfig() throws IOException {
+    close();
+    OzoneConfiguration config = new OzoneConfiguration();
+    config.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2,
+        StorageUnit.KB);
+    int data = 3;
+    int parity = 2;
+    int chunkSize = 1024;
+    createNewClient(config,
+        new MultiNodePipelineBlockAllocator(config, data + parity, 15));
+    String value = new String(new byte[chunkSize], UTF_8);
+    OzoneBucket bucket = getOzoneBucket();
+
+    for (int i = 0; i < 10; i++) {
+      String keyName = UUID.randomUUID().toString();
+      try (OzoneOutputStream out = bucket
+          .createKey(keyName, value.getBytes(UTF_8).length,
+              new ECReplicationConfig(data, parity,
+                  ECReplicationConfig.EcCodec.RS, chunkSize),
+              new HashMap<>())) {
+        out.write(value.getBytes(UTF_8));
+        out.write(value.getBytes(UTF_8));
+      }
+      OzoneKey key = bucket.getKey(keyName);
+      Assert.assertEquals(keyName, key.getName());
+    }
+  }
+
   private OzoneBucket getOzoneBucket() throws IOException {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneECClient.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneECClient.java
new file mode 100644
index 0000000..38b0940
--- /dev/null
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneECClient.java
@@ -0,0 +1,1066 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry;
+import org.apache.hadoop.ozone.client.io.BlockStreamAccessor;
+import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.protocolPB.OmTransport;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.ozone.erasurecode.rawcoder.RSRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * Real unit test for OzoneECClient.
+ * <p>
+ * Used for testing Ozone client without external network calls.
+ */
+public class TestOzoneECClient {
+  private int chunkSize = 1024;
+  private int dataBlocks = 3;
+  private int parityBlocks = 2;
+  private int inputSize = chunkSize * dataBlocks;
+  private OzoneClient client;
+  private ObjectStore store;
+  private String keyName = UUID.randomUUID().toString();
+  private String volumeName = UUID.randomUUID().toString();
+  private String bucketName = UUID.randomUUID().toString();
+  private byte[][] inputChunks = new byte[dataBlocks][chunkSize];
+  private final XceiverClientFactory factoryStub =
+      new MockXceiverClientFactory();
+  private OzoneConfiguration conf = new OzoneConfiguration();
+  private MultiNodePipelineBlockAllocator allocator =
+      new MultiNodePipelineBlockAllocator(conf, dataBlocks + parityBlocks, 15);
+  private final MockOmTransport transportStub = new MockOmTransport(allocator);
+  private final RawErasureEncoder encoder =
+      new RSRawErasureCoderFactory().createEncoder(
+          new ECReplicationConfig(dataBlocks, parityBlocks));
+
+  @Before
+  public void init() throws IOException {
+    conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2,
+        StorageUnit.KB);
+    createNewClient(conf, transportStub);
+  }
+
+  private void createNewClient(ConfigurationSource config,
+      MockBlockAllocator blkAllocator) throws IOException {
+    createNewClient(config, new MockOmTransport(blkAllocator));
+  }
+
+  private void createNewClient(ConfigurationSource config,
+      final MockOmTransport transport) throws IOException {
+    client = new OzoneClient(config, new RpcClient(config, null) {
+
+      @Override
+      protected OmTransport createOmTransport(String omServiceId)
+          throws IOException {
+        return transport;
+      }
+
+      @Override
+      protected XceiverClientFactory createXceiverClientFactory(
+          List<X509Certificate> x509Certificates) throws IOException {
+        return factoryStub;
+      }
+    });
+
+    store = client.getObjectStore();
+    initInputChunks();
+  }
+
+  private void initInputChunks() {
+    for (int i = 0; i < dataBlocks; i++) {
+      inputChunks[i] = getBytesWith(i + 1, chunkSize);
+    }
+  }
+
+  private byte[] getBytesWith(int singleDigitNumber, int total) {
+    StringBuilder builder = new StringBuilder(singleDigitNumber);
+    for (int i = 1; i <= total; i++) {
+      builder.append(singleDigitNumber);
+    }
+    return builder.toString().getBytes(UTF_8);
+  }
+
+  @After
+  public void close() throws IOException {
+    client.close();
+  }
+
+  @Test
+  public void testPutECKeyAndCheckDNStoredData() throws IOException {
+    OzoneBucket bucket = writeIntoECKey(inputChunks, keyName, null);
+    OzoneKey key = bucket.getKey(keyName);
+    Assert.assertEquals(keyName, key.getName());
+    Map<DatanodeDetails, MockDatanodeStorage> storages =
+        ((MockXceiverClientFactory) factoryStub).getStorages();
+    DatanodeDetails[] dnDetails =
+        storages.keySet().toArray(new DatanodeDetails[storages.size()]);
+    Arrays.sort(dnDetails);
+    for (int i = 0; i < inputChunks.length; i++) {
+      MockDatanodeStorage datanodeStorage = storages.get(dnDetails[i]);
+      Assert.assertEquals(1, datanodeStorage.getAllBlockData().size());
+      ByteString content =
+          datanodeStorage.getAllBlockData().values().iterator().next();
+      Assert.assertEquals(new String(inputChunks[i], UTF_8),
+          content.toStringUtf8());
+    }
+  }
+
+  @Test
+  public void testPutECKeyAndCheckParityData() throws IOException {
+    OzoneBucket bucket = writeIntoECKey(inputChunks, keyName, null);
+    final ByteBuffer[] dataBuffers = new ByteBuffer[dataBlocks];
+    for (int i = 0; i < inputChunks.length; i++) {
+      dataBuffers[i] = ByteBuffer.wrap(inputChunks[i]);
+    }
+    final ByteBuffer[] parityBuffers = new ByteBuffer[parityBlocks];
+    for (int i = 0; i < parityBlocks; i++) {
+      parityBuffers[i] = ByteBuffer.allocate(chunkSize);
+    }
+    encoder.encode(dataBuffers, parityBuffers);
+    OzoneKey key = bucket.getKey(keyName);
+    Assert.assertEquals(keyName, key.getName());
+    Map<DatanodeDetails, MockDatanodeStorage> storages =
+        ((MockXceiverClientFactory) factoryStub).getStorages();
+    DatanodeDetails[] dnDetails =
+        storages.keySet().toArray(new DatanodeDetails[storages.size()]);
+    Arrays.sort(dnDetails);
+
+    for (int i = dataBlocks; i < parityBlocks + dataBlocks; i++) {
+      MockDatanodeStorage datanodeStorage = storages.get(dnDetails[i]);
+      Assert.assertEquals(1, datanodeStorage.getAllBlockData().size());
+      ByteString content =
+          datanodeStorage.getAllBlockData().values().iterator().next();
+      Assert.assertEquals(
+          new String(parityBuffers[i - dataBlocks].array(), UTF_8),
+          content.toStringUtf8());
+    }
+
+  }
+
+  @Test
+  public void testPutECKeyAndReadContent() throws IOException {
+    OzoneBucket bucket = writeIntoECKey(inputChunks, keyName, null);
+    OzoneKey key = bucket.getKey(keyName);
+    Assert.assertEquals(keyName, key.getName());
+    try (OzoneInputStream is = bucket.readKey(keyName)) {
+      byte[] fileContent = new byte[chunkSize];
+      for (int i = 0; i < dataBlocks; i++) {
+        Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+        Assert.assertTrue(Arrays.equals(inputChunks[i], fileContent));
+      }
+      // A further read should give EOF
+      Assert.assertEquals(-1, is.read(fileContent));
+    }
+  }
+
+  @Test
+  public void testCreateBucketWithDefaultReplicationConfig()
+      throws IOException {
+    final OzoneBucket bucket = writeIntoECKey(inputChunks, keyName,
+        new DefaultReplicationConfig(ReplicationType.EC,
+            new ECReplicationConfig(dataBlocks, parityBlocks,
+                ECReplicationConfig.EcCodec.RS, chunkSize)));
+
+    // create key without mentioning replication config. Since we set EC
+    // replication in bucket, key should be EC key.
+    try (OzoneOutputStream out = bucket.createKey("mykey", inputSize)) {
+      Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+      for (int i = 0; i < inputChunks.length; i++) {
+        out.write(inputChunks[i]);
+      }
+    }
+  }
+
+  @Test
+  public void test4ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(4);
+  }
+
+  // Test random number of chunks in single write op.
+  @Test
+  public void test5ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(5);
+  }
+
+  @Test
+  public void test6ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(6);
+  }
+
+  @Test
+  public void test7ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(7);
+  }
+
+  @Test
+  public void test9ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(9);
+  }
+
+  @Test
+  public void test10ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(10);
+  }
+
+  @Test
+  public void test12ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(12);
+  }
+
+  public void testMultipleChunksInSingleWriteOp(int numChunks)
+      throws IOException {
+    byte[] inputData = new byte[numChunks * chunkSize];
+    for (int i = 0; i < numChunks; i++) {
+      int start = (i * chunkSize);
+      Arrays.fill(inputData, start, start + chunkSize - 1,
+          String.valueOf(i % 9).getBytes(UTF_8)[0]);
+    }
+    final OzoneBucket bucket = writeIntoECKey(inputData, keyName,
+        new DefaultReplicationConfig(ReplicationType.EC,
+            new ECReplicationConfig(dataBlocks, parityBlocks,
+                ECReplicationConfig.EcCodec.RS, chunkSize)));
+    OzoneKey key = bucket.getKey(keyName);
+    validateContent(inputData, bucket, key);
+  }
+
+  private void validateContent(byte[] inputData, OzoneBucket bucket,
+      OzoneKey key) throws IOException {
+    Assert.assertEquals(keyName, key.getName());
+    try (OzoneInputStream is = bucket.readKey(keyName)) {
+      byte[] fileContent = new byte[inputData.length];
+      Assert.assertEquals(inputData.length, is.read(fileContent));
+      Assert.assertEquals(new String(inputData, UTF_8),
+          new String(fileContent, UTF_8));
+    }
+  }
+
+  @Test
+  public void testSmallerThanChunkSize() throws IOException {
+    byte[] firstSmallChunk = new byte[chunkSize - 1];
+    Arrays.fill(firstSmallChunk, 0, firstSmallChunk.length - 1,
+        Byte.parseByte("1"));
+
+    writeIntoECKey(firstSmallChunk, keyName,
+        new DefaultReplicationConfig(ReplicationType.EC,
+            new ECReplicationConfig(dataBlocks, parityBlocks,
+                ECReplicationConfig.EcCodec.RS, chunkSize)));
+    OzoneManagerProtocolProtos.KeyLocationList blockList =
+        transportStub.getKeys().get(volumeName).get(bucketName).get(keyName)
+            .getKeyLocationListList().get(0);
+
+    Map<DatanodeDetails, MockDatanodeStorage> storages =
+        ((MockXceiverClientFactory) factoryStub).getStorages();
+    OzoneManagerProtocolProtos.KeyLocation keyLocations =
+        blockList.getKeyLocations(0);
+
+    List<MockDatanodeStorage> dns = new ArrayList<>();
+    for (int i = 0; i < dataBlocks + parityBlocks; i++) {
+      HddsProtos.DatanodeDetailsProto member =
+          blockList.getKeyLocations(0).getPipeline().getMembers(i);
+      MockDatanodeStorage mockDatanodeStorage =
+          storages.get(getMatchingStorage(storages, member.getUuid()));
+      dns.add(mockDatanodeStorage);
+    }
+    String firstBlockData = dns.get(0).getFullBlockData(new BlockID(
+        keyLocations.getBlockID().getContainerBlockID().getContainerID(),
+        keyLocations.getBlockID().getContainerBlockID().getLocalID()));
+
+    Assert.assertArrayEquals(firstSmallChunk, firstBlockData.getBytes(UTF_8));
+
+    final ByteBuffer[] dataBuffers = new ByteBuffer[dataBlocks];
+    dataBuffers[0] = ByteBuffer.wrap(firstSmallChunk);
+    //Let's pad the remaining length equal to firstSmall chunk len
+    for (int i = 1; i < dataBlocks; i++) {
+      dataBuffers[i] = ByteBuffer.allocate(firstSmallChunk.length);
+      Arrays.fill(dataBuffers[i].array(), 0, firstSmallChunk.length, (byte) 0);
+    }
+
+    final ByteBuffer[] parityBuffers = new ByteBuffer[parityBlocks];
+    for (int i = 0; i < parityBlocks; i++) {
+      parityBuffers[i] = ByteBuffer.allocate(firstSmallChunk.length);
+    }
+    encoder.encode(dataBuffers, parityBuffers);
+
+    //Lets assert the parity data.
+    for (int i = dataBlocks; i < dataBlocks + parityBlocks; i++) {
+      String parityBlockData = dns.get(i).getFullBlockData(new BlockID(
+          keyLocations.getBlockID().getContainerBlockID().getContainerID(),
+          keyLocations.getBlockID().getContainerBlockID().getLocalID()));
+      String expected =
+          new String(parityBuffers[i - dataBlocks].array(), UTF_8);
+      Assert.assertEquals(expected, parityBlockData);
+      Assert.assertEquals(expected.length(), parityBlockData.length());
+
+    }
+  }
+
+  private static DatanodeDetails getMatchingStorage(
+      Map<DatanodeDetails, MockDatanodeStorage> storages, String uuid) {
+    Iterator<DatanodeDetails> iterator = storages.keySet().iterator();
+    while (iterator.hasNext()) {
+      DatanodeDetails dn = iterator.next();
+      if (dn.getUuid().toString().equals(uuid)) {
+        return dn;
+      }
+    }
+    return null;
+  }
+
+  @Test
+  public void testMultipleChunksWithPartialChunkInSingleWriteOp()
+      throws IOException {
+    final int partialChunkLen = 10;
+    final int numFullChunks = 9;
+    final int inputBuffLen = (numFullChunks * chunkSize) + partialChunkLen;
+    byte[] inputData = new byte[inputBuffLen];
+    for (int i = 0; i < numFullChunks; i++) {
+      int start = (i * chunkSize);
+      Arrays.fill(inputData, start, start + chunkSize - 1,
+          String.valueOf(i).getBytes(UTF_8)[0]);
+    }
+    //fill the last partial chunk as well.
+    Arrays.fill(inputData, (numFullChunks * chunkSize),
+        ((numFullChunks * chunkSize)) + partialChunkLen - 1, (byte) 1);
+    final OzoneBucket bucket = writeIntoECKey(inputData, keyName,
+        new DefaultReplicationConfig(ReplicationType.EC,
+            new ECReplicationConfig(dataBlocks, parityBlocks,
+                ECReplicationConfig.EcCodec.RS, chunkSize)));
+    OzoneKey key = bucket.getKey(keyName);
+    validateContent(inputData, bucket, key);
+  }
+
+  @Test
+  public void testCommitKeyInfo()
+      throws IOException {
+    final OzoneBucket bucket = writeIntoECKey(inputChunks, keyName,
+        new DefaultReplicationConfig(ReplicationType.EC,
+            new ECReplicationConfig(dataBlocks, parityBlocks,
+                ECReplicationConfig.EcCodec.RS, chunkSize)));
+
+    // create key without mentioning replication config. Since we set EC
+    // replication in bucket, key should be EC key.
+    try (OzoneOutputStream out = bucket.createKey("mykey", 6 * inputSize)) {
+      Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+      // Block Size is 2kb, so to create 3 blocks we need 6 iterations here
+      for (int j = 0; j < 6; j++) {
+        for (int i = 0; i < inputChunks.length; i++) {
+          out.write(inputChunks[i]);
+        }
+      }
+    }
+    OzoneManagerProtocolProtos.KeyLocationList blockList =
+        transportStub.getKeys().get(volumeName).get(bucketName).get("mykey")
+            .getKeyLocationListList().get(0);
+
+    Assert.assertEquals(3, blockList.getKeyLocationsCount());
+    // As the mock allocator allocates block with id's increasing sequentially
+    // from 1. Therefore the block should be in the order with id starting 1, 2
+    // and then 3.
+    for (int i = 0; i < 3; i++) {
+      long localId = blockList.getKeyLocationsList().get(i).getBlockID()
+          .getContainerBlockID().getLocalID();
+      Assert.assertEquals(i + 1, localId);
+    }
+
+    Assert.assertEquals(1,
+        transportStub.getKeys().get(volumeName).get(bucketName).get("mykey")
+            .getKeyLocationListCount());
+    Assert.assertEquals(inputChunks[0].length * 3 * 6,
+        transportStub.getKeys().get(volumeName).get(bucketName).get("mykey")
+            .getDataSize());
+  }
+
+  @Test
+  public void testPartialStripeWithSingleChunkAndPadding() throws IOException {
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    try (OzoneOutputStream out = bucket.createKey(keyName, inputSize,
+        new ECReplicationConfig(dataBlocks, parityBlocks,
+            ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+      for (int i = 0; i < inputChunks[0].length; i++) {
+        out.write(inputChunks[0][i]);
+      }
+    }
+    OzoneKey key = bucket.getKey(keyName);
+    validateContent(inputChunks[0], bucket, key);
+  }
+
+  @Test
+  public void testPartialStripeLessThanSingleChunkWithPadding()
+      throws IOException {
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    try (OzoneOutputStream out = bucket.createKey(keyName, inputSize,
+        new ECReplicationConfig(dataBlocks, parityBlocks,
+            ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+      for (int i = 0; i < inputChunks[0].length - 1; i++) {
+        out.write(inputChunks[0][i]);
+      }
+    }
+    OzoneKey key = bucket.getKey(keyName);
+    validateContent(Arrays.copyOf(inputChunks[0], inputChunks[0].length - 1),
+        bucket, key);
+  }
+
+  @Test
+  public void testPartialStripeWithPartialLastChunk()
+      throws IOException {
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    // Last chunk is one byte short of the others.
+    byte[] lastChunk =
+        Arrays.copyOf(inputChunks[inputChunks.length - 1],
+            inputChunks[inputChunks.length - 1].length - 1);
+
+    int inSize = chunkSize * (inputChunks.length - 1) + lastChunk.length;
+    try (OzoneOutputStream out = bucket.createKey(keyName, inSize,
+        new ECReplicationConfig(dataBlocks, parityBlocks,
+            ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+      for (int i = 0; i < inputChunks.length - 1; i++) {
+        out.write(inputChunks[i]);
+      }
+
+      for (int i = 0; i < lastChunk.length; i++) {
+        out.write(lastChunk[i]);
+      }
+    }
+
+    try (OzoneInputStream is = bucket.readKey(keyName)) {
+      byte[] fileContent = new byte[chunkSize];
+      for (int i = 0; i < 2; i++) {
+        Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+        Assert.assertTrue(Arrays.equals(inputChunks[i], fileContent));
+      }
+      Assert.assertEquals(lastChunk.length, is.read(fileContent));
+      Assert.assertTrue(Arrays.equals(lastChunk,
+          Arrays.copyOf(fileContent, lastChunk.length)));
+      // A further read should give EOF
+      Assert.assertEquals(-1, is.read(fileContent));
+    }
+  }
+
+  @Test
+  public void test10D4PConfigWithPartialStripe()
+      throws IOException {
+    // A large block size try to trigger potential overflow
+    // refer to: HDDS-6295
+    conf.set(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
+        OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT);
+    int dataBlks = 10;
+    int parityBlks = 4;
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    // A partial chunk to trigger partialStripe check
+    // in ECKeyOutputStream.close()
+    int inSize = chunkSize - 1;
+    byte[] partialChunk = new byte[inSize];
+
+    try (OzoneOutputStream out = bucket.createKey(keyName, inSize,
+        new ECReplicationConfig(dataBlks, parityBlks,
+            ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+      out.write(partialChunk);
+    }
+
+    try (OzoneInputStream is = bucket.readKey(keyName)) {
+      byte[] fileContent = new byte[chunkSize];
+      Assert.assertEquals(inSize, is.read(fileContent));
+      Assert.assertTrue(Arrays.equals(partialChunk,
+          Arrays.copyOf(fileContent, inSize)));
+    }
+  }
+
+  @Test
+  public void testWriteShouldFailIfMoreThanParityNodesFail()
+      throws IOException {
+    testNodeFailuresWhileWriting(new int[] {0, 1, 2}, 3, 2);
+  }
+
+  @Test
+  public void testWriteShouldSuccessIfLessThanParityNodesFail()
+      throws IOException {
+    testNodeFailuresWhileWriting(new int[] {0}, 2, 2);
+  }
+
+  @Test
+  public void testWriteShouldSuccessIf4NodesFailed() throws IOException {
+    testNodeFailuresWhileWriting(new int[] {0, 1, 2, 3}, 1, 2);
+  }
+
+  @Test
+  public void testWriteShouldSuccessWithAdditional1BlockGroupAfterFailure()
+      throws IOException {
+    testNodeFailuresWhileWriting(new int[] {0, 1, 2, 3}, 10, 3);
+  }
+
+  @Test
+  public void testStripeWriteRetriesOn2Failures() throws IOException {
+    OzoneConfiguration con = new OzoneConfiguration();
+    con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2, StorageUnit.KB);
+    // Cluster has 15 nodes. So, first we will create 3 block groups with
+    // distinct nodes in each. Block Group 1:  0-4, Block Group 2: 5-9, Block
+    // Group 3: 10-14
+    // To mark the node failed in the second block group.
+    int[] nodesIndexesToMarkFailure = new int[2];
+    nodesIndexesToMarkFailure[0] = 0;
+    // To mark the node failed in the second block group also.
+    nodesIndexesToMarkFailure[1] = 5;
+    // Mocked MultiNodePipelineBlockAllocator#allocateBlock implementation
+    // should pick next good block group as we have 15 nodes.
+    int clusterSize = 15;
+    testStripeWriteRetriesOnFailures(con, clusterSize,
+        nodesIndexesToMarkFailure);
+    // It should have used 3rd block group also. So, total initialized nodes
+    // count should be clusterSize.
+    Assert.assertTrue(((MockXceiverClientFactory) factoryStub).getStorages()
+        .size() == clusterSize);
+  }
+
+  @Test
+  public void testStripeWriteRetriesOn3Failures() throws IOException {
+    OzoneConfiguration con = new OzoneConfiguration();
+    con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2, StorageUnit.KB);
+
+    int[] nodesIndexesToMarkFailure = new int[3];
+    nodesIndexesToMarkFailure[0] = 0;
+    // To mark the node failed in the second block group.
+    nodesIndexesToMarkFailure[1] = 5;
+    // To mark the node failed in the third block group.
+    nodesIndexesToMarkFailure[2] = 10;
+    // Mocked MultiNodePipelineBlockAllocator#allocateBlock implementation will
+    // pick the remaining goods for the next block group.
+    int clusterSize = 15;
+    testStripeWriteRetriesOnFailures(con, clusterSize,
+        nodesIndexesToMarkFailure);
+    // It should have used 3rd block group also. So, total initialized nodes
+    // count should be clusterSize.
+    Assert.assertTrue(((MockXceiverClientFactory) factoryStub).getStorages()
+        .size() == clusterSize);
+  }
+
+  // The mocked impl throws IllegalStateException when there are not enough
+  // nodes in allocateBlock request. But write() converts it to IOException.
+  @Test(expected = IOException.class)
+  public void testStripeWriteRetriesOnAllNodeFailures() throws IOException {
+    OzoneConfiguration con = new OzoneConfiguration();
+    con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2, StorageUnit.KB);
+
+    // After writing first stripe, we will mark all nodes as bad in the cluster.
+    int clusterSize = 5;
+    int[] nodesIndexesToMarkFailure = new int[clusterSize];
+    for (int i = 0; i < nodesIndexesToMarkFailure.length; i++) {
+      nodesIndexesToMarkFailure[i] = i;
+    }
+    // Mocked MultiNodePipelineBlockAllocator#allocateBlock implementation can
+    // not pick new block group as all nodes in cluster marked as bad.
+    testStripeWriteRetriesOnFailures(con, clusterSize,
+        nodesIndexesToMarkFailure);
+  }
+
+  @Test
+  public void testStripeWriteRetriesOn4FailuresWith3RetriesAllowed()
+      throws IOException {
+    OzoneConfiguration con = new OzoneConfiguration();
+    con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2, StorageUnit.KB);
+    con.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES, 3);
+
+    int[] nodesIndexesToMarkFailure = new int[4];
+    nodesIndexesToMarkFailure[0] = 0;
+    //To mark node failed in second block group.
+    nodesIndexesToMarkFailure[1] = 5;
+    //To mark node failed in third block group.
+    nodesIndexesToMarkFailure[2] = 10;
+    //To mark node failed in fourth block group.
+    nodesIndexesToMarkFailure[3] = 15;
+    try {
+      // Mocked MultiNodePipelineBlockAllocator#allocateBlock implementation can
+      // pick good block group, but client retries should be limited
+      // OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES_ON_FAILURE(here it was
+      // configured as 3). So, it should fail as we have marked 3 nodes as bad.
+      testStripeWriteRetriesOnFailures(con, 20, nodesIndexesToMarkFailure);
+      Assert.fail(
+          "Expecting it to fail as retries should exceed the max allowed times:"
+              + " " + 3);
+    } catch (IOException e) {
+      Assert.assertEquals("Completed max allowed retries 3 on stripe failures.",
+          e.getMessage());
+    }
+  }
+
+  public void testStripeWriteRetriesOnFailures(OzoneConfiguration con,
+      int clusterSize, int[] nodesIndexesToMarkFailure) throws IOException {
+    close();
+    MultiNodePipelineBlockAllocator blkAllocator =
+        new MultiNodePipelineBlockAllocator(con, dataBlocks + parityBlocks,
+            clusterSize);
+    createNewClient(con, blkAllocator);
+    int numChunksToWriteAfterFailure = 3;
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    try (OzoneOutputStream out = bucket.createKey(keyName, 1024 * 3,
+        new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+            chunkSize), new HashMap<>())) {
+      for (int i = 0; i < dataBlocks; i++) {
+        out.write(inputChunks[i]);
+      }
+      Assert.assertTrue(
+          ((MockXceiverClientFactory) factoryStub).getStorages().size() == 5);
+      List<DatanodeDetails> failedDNs = new ArrayList<>();
+      List<HddsProtos.DatanodeDetailsProto> dns = blkAllocator.getClusterDns();
+
+      for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+        failedDNs.add(DatanodeDetails
+            .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+      }
+
+      // First let's set storage as bad
+      ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+      // Writer should be able to write by using 3rd block group.
+      for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+        out.write(inputChunks[i]);
+      }
+    }
+    final OzoneKeyDetails key = bucket.getKey(keyName);
+    // Data supposed to store in single block group. Since we introduced the
+    // failures after first stripe, the second stripe data should have been
+    // written into new blockgroup. So, we should have 2 block groups. That
+    // means two keyLocations.
+    Assert.assertEquals(2, key.getOzoneKeyLocations().size());
+    try (OzoneInputStream is = bucket.readKey(keyName)) {
+      byte[] fileContent = new byte[chunkSize];
+      for (int i = 0; i < dataBlocks; i++) {
+        Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+        Assert.assertTrue("Expected: " + new String(inputChunks[i],
+                UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+            Arrays.equals(inputChunks[i], fileContent));
+      }
+      for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+        Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+        Assert.assertTrue("Expected: " + new String(inputChunks[i],
+                UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+            Arrays.equals(inputChunks[i], fileContent));
+      }
+    }
+  }
+
+  public void testNodeFailuresWhileWriting(int[] nodesIndexesToMarkFailure,
+      int numChunksToWriteAfterFailure, int numExpectedBlockGrps)
+      throws IOException {
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    try (OzoneOutputStream out = bucket.createKey(keyName, 1024 * 3,
+        new ECReplicationConfig(3, 2,
+            ECReplicationConfig.EcCodec.RS,
+            chunkSize), new HashMap<>())) {
+      for (int i = 0; i < dataBlocks; i++) {
+        out.write(inputChunks[i]);
+      }
+
+      List<DatanodeDetails> failedDNs = new ArrayList<>();
+      List<HddsProtos.DatanodeDetailsProto> dns = allocator.getClusterDns();
+      for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+        failedDNs.add(DatanodeDetails
+            .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+      }
+
+      // First let's set storage as bad
+      ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+      for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+        out.write(inputChunks[i % dataBlocks]);
+      }
+    }
+    final OzoneKeyDetails key = bucket.getKey(keyName);
+    // Data supposed to store in single block group. Since we introduced the
+    // failures after first stripe, the second stripe data should have been
+    // written into new block group. So, we should have numExpectedBlockGrps.
+    // That means two keyLocations.
+    Assert
+        .assertEquals(numExpectedBlockGrps, key.getOzoneKeyLocations().size());
+    try (OzoneInputStream is = bucket.readKey(keyName)) {
+      byte[] fileContent = new byte[chunkSize];
+      for (int i = 0; i < dataBlocks; i++) {
+        Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+        Assert.assertTrue("Expected: " + new String(inputChunks[i],
+                UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+            Arrays.equals(inputChunks[i], fileContent));
+      }
+      for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+        Assert.assertEquals(inputChunks[i % dataBlocks].length,
+            is.read(fileContent));
+        Assert.assertTrue("Expected: " + new String(inputChunks[i % dataBlocks],
+                UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+            Arrays.equals(inputChunks[i % dataBlocks], fileContent));
+      }
+    }
+  }
+
+  @Test
+  public void testLargeWriteOfMultipleStripesWithStripeFailure()
+      throws IOException {
+    close();
+    OzoneConfiguration con = new OzoneConfiguration();
+    // block size of 3KB could hold 3 full stripes
+    con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 3, StorageUnit.KB);
+    con.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES, 3);
+    MultiNodePipelineBlockAllocator blkAllocator =
+        new MultiNodePipelineBlockAllocator(con, dataBlocks + parityBlocks,
+            15);
+    createNewClient(con, blkAllocator);
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    // should write > 1 full stripe to trigger potential issue
+    int numFullStripesBeforeFailure = 2;
+    int numChunksToWriteAfterFailure = dataBlocks;
+    int numExpectedBlockGrps = 2;
+    // fail the DNs for parity blocks
+    int[] nodesIndexesToMarkFailure = {3, 4};
+
+    try (OzoneOutputStream out = bucket.createKey(keyName,
+        1024 * dataBlocks * numFullStripesBeforeFailure
+            + numChunksToWriteAfterFailure,
+        new ECReplicationConfig(dataBlocks, parityBlocks,
+            ECReplicationConfig.EcCodec.RS,
+            chunkSize), new HashMap<>())) {
+      for (int j = 0; j < numFullStripesBeforeFailure; j++) {
+        for (int i = 0; i < dataBlocks; i++) {
+          out.write(inputChunks[i]);
+        }
+      }
+
+      List<DatanodeDetails> failedDNs = new ArrayList<>();
+      List<HddsProtos.DatanodeDetailsProto> dns = allocator.getClusterDns();
+      for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+        failedDNs.add(DatanodeDetails
+            .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+      }
+
+      // First let's set storage as bad
+      ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+      for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+        out.write(inputChunks[i % dataBlocks]);
+      }
+    }
+
+    final OzoneKeyDetails key = bucket.getKey(keyName);
+    // Data supposed to store in single block group. Since we introduced the
+    // failures after first stripe, the second stripe data should have been
+    // written into new block group. So, we should have numExpectedBlockGrps.
+    // That means two keyLocations.
+    Assert
+        .assertEquals(numExpectedBlockGrps, key.getOzoneKeyLocations().size());
+    try (OzoneInputStream is = bucket.readKey(keyName)) {
+      byte[] fileContent = new byte[chunkSize];
+      for (int i = 0; i < dataBlocks * numFullStripesBeforeFailure; i++) {
+        Assert.assertEquals(inputChunks[i % dataBlocks].length,
+            is.read(fileContent));
+        Assert.assertTrue(
+            "Expected: " + new String(inputChunks[i % dataBlocks], UTF_8)
+                + " \n " + "Actual: " + new String(fileContent, UTF_8),
+            Arrays.equals(inputChunks[i % dataBlocks], fileContent));
+      }
+      for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+        Assert.assertEquals(inputChunks[i % dataBlocks].length,
+            is.read(fileContent));
+        Assert.assertTrue(
+            "Expected: " + new String(inputChunks[i % dataBlocks],
+                UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+            Arrays.equals(inputChunks[i % dataBlocks], fileContent));
+      }
+    }
+  }
+
+  @Test
+  public void testPartialStripeWithPartialChunkRetry()
+      throws IOException {
+    close();
+    OzoneConfiguration con = new OzoneConfiguration();
+    // block size of 3KB could hold 3 full stripes
+    con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 3, StorageUnit.KB);
+    con.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES, 3);
+    MultiNodePipelineBlockAllocator blkAllocator =
+        new MultiNodePipelineBlockAllocator(con, dataBlocks + parityBlocks, 15);
+    createNewClient(con, blkAllocator);
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+    int numFullChunks = 7;
+    //Prepare additional partial chunk.
+    int partialChunkSize = 1020;
+    byte[] partialChunk = new byte[partialChunkSize];
+    Arrays.fill(partialChunk, 0, partialChunk.length, "1".getBytes(UTF_8)[0]);
+
+    // A partial chunk to trigger partialStripe check
+    // in ECKeyOutputStream.close()
+    int inSize = chunkSize;
+    try (OzoneOutputStream out = bucket.createKey(keyName, inSize,
+        new ECReplicationConfig(dataBlocks, parityBlocks,
+            ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+      for (int i = 0; i < numFullChunks; i++) {
+        out.write(inputChunks[i % dataBlocks]);
+      }
+
+      out.write(partialChunk);
+
+      int[] nodesIndexesToMarkFailure = new int[] {0, 4};
+      List<DatanodeDetails> failedDNs = new ArrayList<>();
+      List<HddsProtos.DatanodeDetailsProto> dns = blkAllocator.getClusterDns();
+      for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+        failedDNs.add(DatanodeDetails
+            .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+      }
+
+      // First let's set storage as bad
+      ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+    }
+
+    try (OzoneInputStream is = bucket.readKey(keyName)) {
+      byte[] fileContent = new byte[chunkSize];
+      for (int i = 0; i < numFullChunks; i++) {
+        Assert.assertEquals(inputChunks[i % dataBlocks].length,
+            is.read(fileContent));
+        Assert.assertTrue("Expected: " + new String(inputChunks[i % dataBlocks],
+                UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+            Arrays.equals(inputChunks[i % dataBlocks], fileContent));
+      }
+
+      byte[] partialChunkToRead = new byte[partialChunkSize];
+      Assert
+          .assertEquals(partialChunkToRead.length, is.read(partialChunkToRead));
+      Assert.assertTrue(Arrays.equals(partialChunk, partialChunkToRead));
+
+      Assert.assertEquals(-1, is.read(partialChunkToRead));
+    }
+  }
+
+  @Test
+  public void testDiscardPreAllocatedBlocksPreventRetryExceeds()
+      throws IOException {
+    close();
+    OzoneConfiguration con = new OzoneConfiguration();
+    int maxRetries = 3;
+    con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
+        2, StorageUnit.KB);
+    con.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES,
+        maxRetries);
+    MultiNodePipelineBlockAllocator blkAllocator =
+        new MultiNodePipelineBlockAllocator(con, dataBlocks + parityBlocks,
+            15);
+    createNewClient(con, blkAllocator);
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    int numStripesBeforeFailure = 1;
+    int numStripesAfterFailure = 1;
+    int numStripesTotal = numStripesBeforeFailure + numStripesAfterFailure;
+    int numExpectedBlockGrps = 2;
+    // fail any DNs to trigger retry
+    int[] nodesIndexesToMarkFailure = {0, 1};
+    long keySize = (long) chunkSize * dataBlocks * numStripesTotal;
+
+    try (OzoneOutputStream out = bucket.createKey(keyName, keySize,
+        new ECReplicationConfig(dataBlocks, parityBlocks,
+            ECReplicationConfig.EcCodec.RS,
+            chunkSize), new HashMap<>())) {
+      Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+      ECKeyOutputStream kos = (ECKeyOutputStream) out.getOutputStream();
+      List<OmKeyLocationInfo> blockInfos = getAllLocationInfoList(kos);
+      Assert.assertEquals(1, blockInfos.size());
+
+      // Mock some pre-allocated blocks to the key,
+      // should be > maxRetries
+      int numPreAllocatedBlocks = maxRetries + 1;
+      BlockID blockID = blockInfos.get(0).getBlockID();
+      Pipeline pipeline = blockInfos.get(0).getPipeline();
+      List<OmKeyLocationInfo> omKeyLocationInfos = new ArrayList<>();
+      for (int i = 0; i < numPreAllocatedBlocks; i++) {
+        BlockID nextBlockID = new BlockID(blockID.getContainerID(),
+            blockID.getLocalID() + i + 1);
+        omKeyLocationInfos.add(new OmKeyLocationInfo.Builder()
+            .setBlockID(nextBlockID)
+            .setPipeline(pipeline)
+            .build());
+      }
+      OmKeyLocationInfoGroup omKeyLocationInfoGroup =
+          new OmKeyLocationInfoGroup(0, omKeyLocationInfos);
+      kos.addPreallocateBlocks(omKeyLocationInfoGroup, 0);
+
+      // Good writes
+      for (int j = 0; j < numStripesBeforeFailure; j++) {
+        for (int i = 0; i < dataBlocks; i++) {
+          out.write(inputChunks[i]);
+        }
+      }
+
+      // Make the writes fail to trigger retry
+      List<DatanodeDetails> failedDNs = new ArrayList<>();
+      List<HddsProtos.DatanodeDetailsProto> dns = allocator.getClusterDns();
+      for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+        failedDNs.add(DatanodeDetails
+            .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+      }
+      // First let's set storage as bad
+      ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+      // Writes that will retry due to failed DNs
+      try {
+        for (int j = 0; j < numStripesAfterFailure; j++) {
+          for (int i = 0; i < dataBlocks; i++) {
+            out.write(inputChunks[i]);
+          }
+        }
+      } catch (IOException e) {
+        // If we don't discard pre-allocated blocks,
+        // retries should exceed the maxRetries and write will fail.
+        Assert.fail("Max retries exceeded");
+      }
+    }
+
+    final OzoneKeyDetails key = bucket.getKey(keyName);
+    Assert.assertEquals(numExpectedBlockGrps,
+        key.getOzoneKeyLocations().size());
+
+    try (OzoneInputStream is = bucket.readKey(keyName)) {
+      byte[] fileContent = new byte[chunkSize];
+      for (int i = 0; i < dataBlocks * numStripesTotal; i++) {
+        Assert.assertEquals(inputChunks[i % dataBlocks].length,
+            is.read(fileContent));
+        Assert.assertArrayEquals(
+            "Expected: " + new String(inputChunks[i % dataBlocks], UTF_8)
+                + " \n " + "Actual: " + new String(fileContent, UTF_8),
+            inputChunks[i % dataBlocks], fileContent);
+      }
+    }
+  }
+
+  private OzoneBucket writeIntoECKey(byte[] data, String key,
+      DefaultReplicationConfig defaultReplicationConfig) throws IOException {
+    return writeIntoECKey(new byte[][] {data}, key, defaultReplicationConfig);
+  }
+
+  private OzoneBucket writeIntoECKey(byte[][] chunks, String key,
+      DefaultReplicationConfig defaultReplicationConfig) throws IOException {
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    if (defaultReplicationConfig != null) {
+      final BucketArgs.Builder builder = BucketArgs.newBuilder();
+      builder.setDefaultReplicationConfig(defaultReplicationConfig);
+      volume.createBucket(bucketName, builder.build());
+    } else {
+      volume.createBucket(bucketName);
+    }
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    int size = (int) Arrays.stream(chunks).mapToLong(a -> a.length).sum();
+    try (OzoneOutputStream out = bucket.createKey(key, size,
+        new ECReplicationConfig(dataBlocks, parityBlocks,
+            ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+      for (int i = 0; i < chunks.length; i++) {
+        out.write(chunks[i]);
+      }
+    }
+    return bucket;
+  }
+
+  private List<OmKeyLocationInfo> getAllLocationInfoList(
+      ECKeyOutputStream kos) {
+    List<OmKeyLocationInfo> locationInfoList = new ArrayList<>();
+    for (BlockOutputStreamEntry streamEntry : kos.getStreamEntries()) {
+      BlockStreamAccessor streamAccessor =
+          new BlockStreamAccessor(streamEntry);
+      OmKeyLocationInfo info =
+          new OmKeyLocationInfo.Builder()
+              .setBlockID(streamAccessor.getStreamBlockID())
+              .setLength(streamAccessor.getStreamCurrentPosition())
+              .setOffset(0)
+              .setToken(streamAccessor.getStreamToken())
+              .setPipeline(streamAccessor.getStreamPipeline())
+              .build();
+      locationInfoList.add(info);
+    }
+    return locationInfoList;
+  }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/BlockStreamAccessor.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/BlockStreamAccessor.java
new file mode 100644
index 0000000..d9a7211
--- /dev/null
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/BlockStreamAccessor.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * An accessor helper class to BlockOutputStreamEntry for test.
+ */
+public class BlockStreamAccessor {
+
+  private final BlockOutputStreamEntry streamEntry;
+
+  public BlockStreamAccessor(BlockOutputStreamEntry entry) {
+    this.streamEntry = entry;
+  }
+
+  public BlockID getStreamBlockID() {
+    return streamEntry.getBlockID();
+  }
+
+  public Pipeline getStreamPipeline() {
+    return streamEntry.getPipeline();
+  }
+
+  public Token<OzoneBlockTokenIdentifier> getStreamToken() {
+    return streamEntry.getToken();
+  }
+
+  public long getStreamCurrentPosition() {
+    return streamEntry.getCurrentPosition();
+  }
+}
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockOutputStreamEntry.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockOutputStreamEntry.java
new file mode 100644
index 0000000..4b87ef7
--- /dev/null
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockOutputStreamEntry.java
@@ -0,0 +1,128 @@
+/*
+ * 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.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * {@link ECBlockOutputStreamEntry} tests.
+ */
+public class TestECBlockOutputStreamEntry {
+
+  @Test
+  public void
+      testAcquireDifferentClientForECBlocksOnTheSameHostButDifferentPort()
+      throws IOException {
+    PipelineID randomId = PipelineID.randomId();
+    ReplicationConfig ecReplicationConfig =
+        new ECReplicationConfig("RS-3-2-1024k");
+    DatanodeDetails node1 = aNode("127.0.0.1", "localhost", 2001);
+    DatanodeDetails node2 = aNode("127.0.0.1", "localhost", 2002);
+    DatanodeDetails node3 = aNode("127.0.0.1", "localhost", 2003);
+    DatanodeDetails node4 = aNode("127.0.0.1", "localhost", 2004);
+    DatanodeDetails node5 = aNode("127.0.0.1", "localhost", 2005);
+    List<DatanodeDetails> nodes =
+        Arrays.asList(node1, node2, node3, node4, node5);
+    Pipeline anECPipeline = Pipeline.newBuilder()
+        .setId(randomId)
+        .setReplicationConfig(ecReplicationConfig)
+        .setState(Pipeline.PipelineState.OPEN)
+        .setNodes(nodes)
+        .build();
+    XceiverClientManager manager =
+        new XceiverClientManager(new OzoneConfiguration());
+    HashSet<XceiverClientSpi> clients = new HashSet<>();
+    ECBlockOutputStreamEntry entry = new ECBlockOutputStreamEntry.Builder()
+        .setXceiverClientManager(manager)
+        .setPipeline(anECPipeline)
+        .build();
+    for (int i = 0; i < nodes.size(); i++) {
+      clients.add(
+          manager.acquireClient(
+              entry.createSingleECBlockPipeline(
+                  anECPipeline, nodes.get(i), i
+              )));
+    }
+    assertEquals(5, clients.size());
+  }
+
+  @Test
+  public void
+      testAcquireDifferentClientForECBlocksOnTheSameHostWithSomeOnSamePortAlso()
+      throws IOException {
+    PipelineID randomId = PipelineID.randomId();
+    ReplicationConfig ecReplicationConfig =
+        new ECReplicationConfig("RS-3-2-1024k");
+    DatanodeDetails node1 = aNode("127.0.0.1", "localhost", 2001);
+    DatanodeDetails node2 = aNode("127.0.0.1", "localhost", 2001);
+    DatanodeDetails node3 = aNode("127.0.0.1", "localhost", 2003);
+    DatanodeDetails node4 = aNode("127.0.0.1", "localhost", 2001);
+    DatanodeDetails node5 = aNode("127.0.0.1", "localhost", 2005);
+    List<DatanodeDetails> nodes =
+        Arrays.asList(node1, node2, node3, node4, node5);
+    Pipeline anECPipeline = Pipeline.newBuilder()
+        .setId(randomId)
+        .setReplicationConfig(ecReplicationConfig)
+        .setState(Pipeline.PipelineState.OPEN)
+        .setNodes(nodes)
+        .build();
+    XceiverClientManager manager =
+        new XceiverClientManager(new OzoneConfiguration());
+    HashSet<XceiverClientSpi> clients = new HashSet<>();
+    ECBlockOutputStreamEntry entry = new ECBlockOutputStreamEntry.Builder()
+        .setXceiverClientManager(manager)
+        .setPipeline(anECPipeline)
+        .build();
+    for (int i = 0; i < nodes.size(); i++) {
+      clients.add(
+          manager.acquireClient(
+              entry.createSingleECBlockPipeline(
+                  anECPipeline, nodes.get(i), i
+              )));
+    }
+    assertEquals(3, clients.size());
+    assertEquals(1, clients.stream().filter(c -> c.getRefcount() == 3).count());
+    assertEquals(2, clients.stream().filter(c -> c.getRefcount() == 1).count());
+  }
+
+  private DatanodeDetails aNode(String ip, String hostName, int port) {
+    return DatanodeDetails.newBuilder()
+        .setUuid(UUID.randomUUID())
+        .setIpAddress(ip)
+        .setHostName(hostName)
+        .addPort(
+            DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port))
+        .build();
+  }
+}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
index c38dbc7..5717ca6 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
@@ -19,6 +19,8 @@
 
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.ratis.util.TimeDuration;
 
@@ -257,6 +259,16 @@
   public static final boolean OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT =
       false;
 
+  public static final String OZONE_SERVER_DEFAULT_REPLICATION_KEY =
+      "ozone.server.default.replication";
+  public static final String OZONE_SERVER_DEFAULT_REPLICATION_DEFAULT =
+      ReplicationFactor.THREE.toString();
+
+  public static final String OZONE_SERVER_DEFAULT_REPLICATION_TYPE_KEY =
+      "ozone.server.default.replication.type";
+  public static final String OZONE_SERVER_DEFAULT_REPLICATION_TYPE_DEFAULT =
+      ReplicationType.RATIS.toString();
+
   public static final String OZONE_OM_HA_PREFIX = "ozone.om.ha";
 
   public static final String OZONE_FS_TRASH_INTERVAL_KEY =
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
index 953aa23..168b2c8 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
@@ -242,6 +242,7 @@
     UPDATE_LAYOUT_VERSION_FAILED,
     LAYOUT_FEATURE_FINALIZATION_FAILED,
     PREPARE_FAILED,
-    NOT_SUPPORTED_OPERATION_WHEN_PREPARED
+    NOT_SUPPORTED_OPERATION_WHEN_PREPARED,
+    NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION
   }
 }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
index 9291d33..b23da41 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
@@ -20,6 +20,7 @@
 import java.util.LinkedHashMap;
 import java.util.Map;
 
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.audit.Auditable;
@@ -51,6 +52,7 @@
 
   private long quotaInBytes;
   private long quotaInNamespace;
+  private DefaultReplicationConfig defaultReplicationConfig = null;
   /**
    * Bucket Owner Name.
    */
@@ -129,7 +131,24 @@
   }
 
   /**
+   * Returns Bucket default replication config.
+   * @return
+   */
+  public DefaultReplicationConfig getDefaultReplicationConfig() {
+    return defaultReplicationConfig;
+  }
+
+  /**
+   * Sets the Bucket default replication config.
+   */
+  private void setDefaultReplicationConfig(
+      DefaultReplicationConfig defaultReplicationConfig) {
+    this.defaultReplicationConfig = defaultReplicationConfig;
+  }
+
+  /**
    * Returns Bucket Owner Name.
+   *
    * @return ownerName.
    */
   public String getOwnerName() {
@@ -173,6 +192,7 @@
     private Map<String, String> metadata;
     private long quotaInBytes;
     private long quotaInNamespace;
+    private DefaultReplicationConfig defaultReplicationConfig;
     private String ownerName;
     /**
      * Constructs a builder.
@@ -217,6 +237,12 @@
       return this;
     }
 
+    public Builder setDefaultReplicationConfig(
+        DefaultReplicationConfig defaultRepConfig) {
+      this.defaultReplicationConfig = defaultRepConfig;
+      return this;
+    }
+
     public Builder setOwnerName(String owner) {
       ownerName = owner;
       return this;
@@ -229,8 +255,11 @@
     public OmBucketArgs build() {
       Preconditions.checkNotNull(volumeName);
       Preconditions.checkNotNull(bucketName);
-      return new OmBucketArgs(volumeName, bucketName, isVersionEnabled,
-          storageType, metadata, quotaInBytes, quotaInNamespace, ownerName);
+      OmBucketArgs omBucketArgs =
+          new OmBucketArgs(volumeName, bucketName, isVersionEnabled,
+              storageType, metadata, quotaInBytes, quotaInNamespace, ownerName);
+      omBucketArgs.setDefaultReplicationConfig(defaultReplicationConfig);
+      return omBucketArgs;
     }
   }
 
@@ -253,6 +282,9 @@
     if (quotaInNamespace > 0 || quotaInNamespace == OzoneConsts.QUOTA_RESET) {
       builder.setQuotaInNamespace(quotaInNamespace);
     }
+    if (defaultReplicationConfig != null) {
+      builder.setDefaultReplicationConfig(defaultReplicationConfig.toProto());
+    }
     if (ownerName != null) {
       builder.setOwnerName(ownerName);
     }
@@ -265,16 +297,22 @@
    * @return instance of OmBucketArgs
    */
   public static OmBucketArgs getFromProtobuf(BucketArgs bucketArgs) {
-    return new OmBucketArgs(bucketArgs.getVolumeName(),
-        bucketArgs.getBucketName(),
-        bucketArgs.hasIsVersionEnabled() ?
-            bucketArgs.getIsVersionEnabled() : null,
-        bucketArgs.hasStorageType() ? StorageType.valueOf(
-            bucketArgs.getStorageType()) : null,
-        KeyValueUtil.getFromProtobuf(bucketArgs.getMetadataList()),
-        bucketArgs.getQuotaInBytes(),
-        bucketArgs.getQuotaInNamespace(),
-        bucketArgs.hasOwnerName() ?
-            bucketArgs.getOwnerName() : null);
+    OmBucketArgs omBucketArgs =
+        new OmBucketArgs(bucketArgs.getVolumeName(),
+            bucketArgs.getBucketName(),
+            bucketArgs.hasIsVersionEnabled() ?
+                bucketArgs.getIsVersionEnabled() : null,
+            bucketArgs.hasStorageType() ? StorageType.valueOf(
+                bucketArgs.getStorageType()) : null,
+            KeyValueUtil.getFromProtobuf(bucketArgs.getMetadataList()),
+            bucketArgs.getQuotaInBytes(),
+            bucketArgs.getQuotaInNamespace(),
+            bucketArgs.hasOwnerName() ?
+                bucketArgs.getOwnerName() : null);
+    // OmBucketArgs ctor already has more arguments, so setting the default
+    // replication config separately.
+    omBucketArgs.setDefaultReplicationConfig(
+        new DefaultReplicationConfig(bucketArgs.getDefaultReplicationConfig()));
+    return omBucketArgs;
   }
 }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
index ad81c8b..846b71c 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
@@ -27,7 +27,13 @@
 import java.util.Objects;
 import java.util.stream.Collectors;
 
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.audit.Auditable;
@@ -75,6 +81,11 @@
    */
   private BucketEncryptionKeyInfo bekInfo;
 
+  /**
+   * Optional default replication for bucket.
+   */
+  private DefaultReplicationConfig defaultReplicationConfig;
+
   private final String sourceVolume;
 
   private final String sourceBucket;
@@ -109,8 +120,10 @@
    * @param usedBytes - Bucket Quota Usage in bytes.
    * @param quotaInBytes Bucket quota in bytes.
    * @param quotaInNamespace Bucket quota in counts.
-   * @param bucketLayout Bucket Layout.
+   * @param bucketLayout bucket layout.
    * @param owner owner of the bucket.
+   * @param defaultReplicationConfig default replication config.
+   * @param bucketLayout Bucket Layout.
    */
   @SuppressWarnings("checkstyle:ParameterNumber")
   private OmBucketInfo(String volumeName,
@@ -131,7 +144,8 @@
       long quotaInBytes,
       long quotaInNamespace,
       BucketLayout bucketLayout,
-      String owner) {
+      String owner,
+      DefaultReplicationConfig defaultReplicationConfig) {
     this.volumeName = volumeName;
     this.bucketName = bucketName;
     this.acls = acls;
@@ -151,6 +165,7 @@
     this.quotaInNamespace = quotaInNamespace;
     this.bucketLayout = bucketLayout;
     this.owner = owner;
+    this.defaultReplicationConfig = defaultReplicationConfig;
   }
 
   /**
@@ -250,12 +265,22 @@
 
   /**
    * Returns the Bucket Layout.
+   *
    * @return BucketLayout.
    */
   public BucketLayout getBucketLayout() {
     return bucketLayout;
   }
 
+  /**
+   * Returns bucket EC replication config.
+   *
+   * @return EC replication config.
+   */
+  public DefaultReplicationConfig getDefaultReplicationConfig() {
+    return defaultReplicationConfig;
+  }
+
   public String getSourceVolume() {
     return sourceVolume;
   }
@@ -358,6 +383,10 @@
         acl.getName(), (BitSet) acl.getAclBitSet().clone(),
         acl.getAclScope())));
 
+    if (defaultReplicationConfig != null) {
+      builder.setDefaultReplicationConfig(defaultReplicationConfig.copy());
+    }
+
     return builder.build();
   }
 
@@ -381,7 +410,21 @@
         .setQuotaInBytes(quotaInBytes)
         .setQuotaInNamespace(quotaInNamespace)
         .setBucketLayout(bucketLayout)
-        .setOwner(owner);
+        .setOwner(owner)
+        .setDefaultReplicationConfig(defaultReplicationConfig);
+  }
+
+  public void setDefaultReplicationConfig(ReplicationConfig replicationConfig) {
+    this.defaultReplicationConfig = new DefaultReplicationConfig(
+        ReplicationType.fromProto(replicationConfig.getReplicationType()),
+        replicationConfig
+            .getReplicationType() == HddsProtos.ReplicationType.EC ?
+            null :
+            ReplicationFactor.valueOf(replicationConfig.getRequiredNodes()),
+        replicationConfig
+            .getReplicationType() == HddsProtos.ReplicationType.EC ?
+            ((ECReplicationConfig) replicationConfig) :
+            null);
   }
 
   /**
@@ -407,6 +450,7 @@
     private long quotaInNamespace;
     private BucketLayout bucketLayout;
     private String owner;
+    private DefaultReplicationConfig defaultReplicationConfig;
 
     public Builder() {
       //Default values
@@ -535,6 +579,12 @@
       return this;
     }
 
+    public Builder setDefaultReplicationConfig(
+        DefaultReplicationConfig defaultReplConfig) {
+      this.defaultReplicationConfig = defaultReplConfig;
+      return this;
+    }
+
     /**
      * Constructs the OmBucketInfo.
      * @return instance of OmBucketInfo.
@@ -545,11 +595,11 @@
       Preconditions.checkNotNull(acls);
       Preconditions.checkNotNull(isVersionEnabled);
       Preconditions.checkNotNull(storageType);
-
       return new OmBucketInfo(volumeName, bucketName, acls, isVersionEnabled,
           storageType, creationTime, modificationTime, objectID, updateID,
           metadata, bekInfo, sourceVolume, sourceBucket, usedBytes,
-          usedNamespace, quotaInBytes, quotaInNamespace, bucketLayout, owner);
+          usedNamespace, quotaInBytes, quotaInNamespace, bucketLayout, owner,
+          defaultReplicationConfig);
     }
   }
 
@@ -578,6 +628,9 @@
     if (bekInfo != null && bekInfo.getKeyName() != null) {
       bib.setBeinfo(OMPBHelper.convert(bekInfo));
     }
+    if (defaultReplicationConfig != null) {
+      bib.setDefaultReplicationConfig(defaultReplicationConfig.toProto());
+    }
     if (sourceVolume != null) {
       bib.setSourceVolume(sourceVolume);
     }
@@ -626,6 +679,10 @@
       obib.setBucketLayout(
           BucketLayout.fromProto(bucketInfo.getBucketLayout()));
     }
+    if (bucketInfo.hasDefaultReplicationConfig()) {
+      obib.setDefaultReplicationConfig(
+          OMPBHelper.convert(bucketInfo.getDefaultReplicationConfig()));
+    }
     if (bucketInfo.hasObjectID()) {
       obib.setObjectID(bucketInfo.getObjectID());
     }
@@ -668,6 +725,7 @@
         ", quotaInBytes='" + quotaInBytes + "'" +
         ", quotaInNamespace='" + quotaInNamespace + "'" +
         ", bucketLayout='" + bucketLayout + '\'' +
+        ", defaultReplicationConfig='" + defaultReplicationConfig + '\'' +
         sourceInfo +
         '}';
   }
@@ -696,7 +754,8 @@
         Objects.equals(sourceBucket, that.sourceBucket) &&
         Objects.equals(metadata, that.metadata) &&
         Objects.equals(bekInfo, that.bekInfo) &&
-        Objects.equals(owner, that.owner);
+        Objects.equals(owner, that.owner) &&
+        Objects.equals(defaultReplicationConfig, this.defaultReplicationConfig);
   }
 
   @Override
@@ -725,6 +784,7 @@
         ", quotaInNamespace=" + quotaInNamespace +
         ", bucketLayout=" + bucketLayout +
         ", owner=" + owner +
+        ", defaultReplicationConfig=" + defaultReplicationConfig +
         '}';
   }
 }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
index c3b8620..4b27276 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
@@ -29,6 +29,7 @@
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.client.ContainerBlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -132,6 +133,10 @@
     return dataSize;
   }
 
+  public long getReplicatedSize() {
+    return QuotaUtil.getReplicatedSize(getDataSize(), replicationConfig);
+  }
+
   public void setDataSize(long size) {
     this.dataSize = size;
   }
@@ -346,6 +351,10 @@
     this.parentObjectID = parentObjectID;
   }
 
+  public void setReplicationConfig(ReplicationConfig repConfig) {
+    this.replicationConfig = repConfig;
+  }
+
   /**
    * Builder of OmKeyInfo.
    */
@@ -552,9 +561,14 @@
         .setVolumeName(volumeName)
         .setBucketName(bucketName)
         .setDataSize(dataSize)
-        .setType(replicationConfig.getReplicationType())
-        .setFactor(ReplicationConfig.getLegacyFactor(replicationConfig))
-        .setLatestVersion(latestVersion)
+        .setType(replicationConfig.getReplicationType());
+    if (replicationConfig instanceof ECReplicationConfig) {
+      kb.setEcReplicationConfig(
+          ((ECReplicationConfig) replicationConfig).toProto());
+    } else {
+      kb.setFactor(ReplicationConfig.getLegacyFactor(replicationConfig));
+    }
+    kb.setLatestVersion(latestVersion)
         .addAllKeyLocationList(keyLocations)
         .setCreationTime(creationTime)
         .setModificationTime(modificationTime)
@@ -594,7 +608,8 @@
         .setCreationTime(keyInfo.getCreationTime())
         .setModificationTime(keyInfo.getModificationTime())
         .setReplicationConfig(ReplicationConfig
-                .fromProtoTypeAndFactor(keyInfo.getType(), keyInfo.getFactor()))
+            .fromProto(keyInfo.getType(), keyInfo.getFactor(),
+                keyInfo.getEcReplicationConfig()))
         .addAllMetadata(KeyValueUtil.getFromProtobuf(keyInfo.getMetadataList()))
         .setFileEncryptionInfo(keyInfo.hasFileEncryptionInfo() ?
             OMPBHelper.convert(keyInfo.getFileEncryptionInfo()) : null)
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java
index 41779ab..86da3eb 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java
@@ -16,6 +16,7 @@
  */
 package org.apache.hadoop.ozone.om.helpers;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartKeyInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
@@ -200,10 +201,11 @@
     multipartKeyInfo.getPartKeyInfoListList().forEach(partKeyInfo ->
         list.put(partKeyInfo.getPartNumber(), partKeyInfo));
 
-    final ReplicationConfig replicationConfig = ReplicationConfig
-            .fromProtoTypeAndFactor(
-                    multipartKeyInfo.getType(),
-                    multipartKeyInfo.getFactor());
+    final ReplicationConfig replicationConfig = ReplicationConfig.fromProto(
+        multipartKeyInfo.getType(),
+        multipartKeyInfo.getFactor(),
+        multipartKeyInfo.getEcReplicationConfig()
+    );
 
     return new OmMultipartKeyInfo(multipartKeyInfo.getUploadID(),
         multipartKeyInfo.getCreationTime(), replicationConfig,
@@ -220,10 +222,17 @@
         .setUploadID(uploadID)
         .setCreationTime(creationTime)
         .setType(replicationConfig.getReplicationType())
-        .setFactor(ReplicationConfig.getLegacyFactor(replicationConfig))
         .setObjectID(objectID)
         .setUpdateID(updateID)
         .setParentID(parentID);
+
+    if (replicationConfig instanceof ECReplicationConfig) {
+      ECReplicationConfig ecConf = (ECReplicationConfig) replicationConfig;
+      builder.setEcReplicationConfig(ecConf.toProto());
+    } else {
+      builder.setFactor(ReplicationConfig.getLegacyFactor(replicationConfig));
+    }
+
     partKeyInfoList.forEach((key, value) -> builder.addPartKeyInfoList(value));
     return builder.build();
   }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.java
new file mode 100644
index 0000000..09e0440
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om.helpers;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Helper class to calculate quota related usage.
+ */
+public final class QuotaUtil {
+
+  private QuotaUtil() {
+  };
+
+  /**
+   * From the used space and replicationConfig, calculate the expected
+   * replicated size of the data.
+   * @param dataSize The number of bytes of data stored
+   * @param repConfig The replicationConfig used to store the data
+   * @return Number of bytes required to store the dataSize with replication
+   */
+  public static long getReplicatedSize(
+      long dataSize, ReplicationConfig repConfig) {
+    if (repConfig.getReplicationType() == RATIS) {
+      return dataSize * ((RatisReplicationConfig)repConfig)
+          .getReplicationFactor().getNumber();
+    } else if (repConfig.getReplicationType() == EC) {
+      ECReplicationConfig rc = (ECReplicationConfig)repConfig;
+      int dataStripeSize = rc.getData() * rc.getEcChunkSize();
+      long fullStripes = dataSize / dataStripeSize;
+      long partialFirstChunk =
+          Math.min(rc.getEcChunkSize(), dataSize % dataStripeSize);
+      long replicationOverhead =
+          fullStripes * rc.getParity() * rc.getEcChunkSize()
+              + partialFirstChunk * rc.getParity();
+      return dataSize + replicationOverhead;
+    } else {
+      return dataSize;
+    }
+  }
+
+}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
index 29d5a09..10ca155 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
@@ -25,6 +25,7 @@
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos
     .UpgradeFinalizationStatus;
@@ -220,7 +221,6 @@
    * @param cmdType type of the request
    */
   private OMRequest.Builder createOMRequest(Type cmdType) {
-
     return OMRequest.newBuilder()
         .setCmdType(cmdType)
         .setVersion(ClientVersion.CURRENT_VERSION)
@@ -618,8 +618,13 @@
     }
 
     if (args.getReplicationConfig() != null) {
-      keyArgs.setFactor(
-          ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+      if (args.getReplicationConfig() instanceof ECReplicationConfig) {
+        keyArgs.setEcReplicationConfig(
+            ((ECReplicationConfig) args.getReplicationConfig()).toProto());
+      } else {
+        keyArgs.setFactor(
+            ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+      }
       keyArgs.setType(args.getReplicationConfig().getReplicationType());
     }
 
@@ -676,8 +681,13 @@
         .setDataSize(args.getDataSize());
 
     if (args.getReplicationConfig() != null) {
-      keyArgs.setFactor(
-          ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+      if (args.getReplicationConfig() instanceof ECReplicationConfig) {
+        keyArgs.setEcReplicationConfig(
+            ((ECReplicationConfig) args.getReplicationConfig()).toProto());
+      } else {
+        keyArgs.setFactor(
+            ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+      }
       keyArgs.setType(args.getReplicationConfig().getReplicationType());
     }
 
@@ -711,8 +721,13 @@
             .collect(Collectors.toList()));
 
     if (args.getReplicationConfig() != null) {
-      keyArgsBuilder.setFactor(
-          ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+      if (args.getReplicationConfig() instanceof ECReplicationConfig) {
+        keyArgsBuilder.setEcReplicationConfig(
+            ((ECReplicationConfig) args.getReplicationConfig()).toProto());
+      } else {
+        keyArgsBuilder.setFactor(
+            ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+      }
       keyArgsBuilder.setType(args.getReplicationConfig().getReplicationType());
     }
 
@@ -948,9 +963,13 @@
             OzoneAcl.toProtobuf(a)).collect(Collectors.toList()));
 
     if (omKeyArgs.getReplicationConfig() != null) {
-      keyArgs.setFactor(
-              ReplicationConfig
-                      .getLegacyFactor(omKeyArgs.getReplicationConfig()));
+      if (omKeyArgs.getReplicationConfig() instanceof ECReplicationConfig) {
+        keyArgs.setEcReplicationConfig(
+            ((ECReplicationConfig) omKeyArgs.getReplicationConfig()).toProto());
+      } else {
+        keyArgs.setFactor(ReplicationConfig
+            .getLegacyFactor(omKeyArgs.getReplicationConfig()));
+      }
       keyArgs.setType(omKeyArgs.getReplicationConfig().getReplicationType());
     }
 
@@ -1085,8 +1104,9 @@
 
     OmMultipartUploadListParts omMultipartUploadListParts =
         new OmMultipartUploadListParts(
-                ReplicationConfig.fromProtoTypeAndFactor(response.getType(),
-                response.getFactor()),
+            ReplicationConfig.fromProto(
+                response.getType(), response.getFactor(),
+                response.getEcReplicationConfig()),
             response.getNextPartNumberMarker(), response.getIsTruncated());
     omMultipartUploadListParts.addProtoPartList(response.getPartsListList());
 
@@ -1121,8 +1141,8 @@
                 proto.getKeyName(),
                 proto.getUploadId(),
                 Instant.ofEpochMilli(proto.getCreationTime()),
-                ReplicationConfig.fromProtoTypeAndFactor(proto.getType(),
-                        proto.getFactor())
+                ReplicationConfig.fromProto(proto.getType(), proto.getFactor(),
+                    proto.getEcReplicationConfig())
             ))
             .collect(Collectors.toList());
 
@@ -1532,9 +1552,13 @@
         .addAllAcls(args.getAcls().stream().map(a ->
             OzoneAcl.toProtobuf(a)).collect(Collectors.toList()));
     if (args.getReplicationConfig() != null) {
-      keyArgsBuilder.setFactor(
-              ReplicationConfig
-                      .getLegacyFactor(args.getReplicationConfig()));
+      if (args.getReplicationConfig() instanceof ECReplicationConfig) {
+        keyArgsBuilder.setEcReplicationConfig(
+            ((ECReplicationConfig) args.getReplicationConfig()).toProto());
+      } else {
+        keyArgsBuilder.setFactor(
+            ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+      }
       keyArgsBuilder.setType(args.getReplicationConfig().getReplicationType());
     }
     CreateFileRequest createFileRequest = CreateFileRequest.newBuilder()
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
index 810111d..a7f1307 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
@@ -21,6 +21,11 @@
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
@@ -142,6 +147,55 @@
         ezKeyVersionName);
   }
 
+  public static DefaultReplicationConfig convert(
+      HddsProtos.DefaultReplicationConfig defaultReplicationConfig) {
+    if (defaultReplicationConfig == null) {
+      throw new IllegalArgumentException(
+          "Invalid argument: default replication config" + " is null");
+    }
+
+    final ReplicationType type =
+        ReplicationType.fromProto(defaultReplicationConfig.getType());
+    DefaultReplicationConfig defaultReplicationConfigObj = null;
+    switch (type) {
+    case EC:
+      defaultReplicationConfigObj = new DefaultReplicationConfig(type,
+          new ECReplicationConfig(
+              defaultReplicationConfig.getEcReplicationConfig()));
+      break;
+    default:
+      final ReplicationFactor factor =
+          ReplicationFactor.fromProto(defaultReplicationConfig.getFactor());
+      defaultReplicationConfigObj = new DefaultReplicationConfig(type, factor);
+    }
+    return defaultReplicationConfigObj;
+  }
+
+  public static HddsProtos.DefaultReplicationConfig convert(
+      DefaultReplicationConfig defaultReplicationConfig) {
+    if (defaultReplicationConfig == null) {
+      throw new IllegalArgumentException(
+          "Invalid argument: default replication config" + " is null");
+    }
+
+    final HddsProtos.DefaultReplicationConfig.Builder builder =
+        HddsProtos.DefaultReplicationConfig.newBuilder();
+    builder.setType(
+        ReplicationType.toProto(defaultReplicationConfig.getType()));
+
+    if (defaultReplicationConfig.getFactor() != null) {
+      builder.setFactor(ReplicationFactor
+          .toProto(defaultReplicationConfig.getFactor()));
+    }
+
+    if (defaultReplicationConfig.getEcReplicationConfig() != null) {
+      builder.setEcReplicationConfig(
+          defaultReplicationConfig.getEcReplicationConfig().toProto());
+    }
+
+    return builder.build();
+  }
+
   public static CipherSuite convert(CipherSuiteProto proto) {
     switch (proto) {
     case AES_CTR_NOPADDING:
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java
index 650fc91..aa64fd2 100644
--- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.ozone.om.helpers;
 
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.protocol.StorageType;
 
 import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.junit.Assert;
 import org.junit.Test;
@@ -111,4 +115,56 @@
     Assert.assertEquals((int) 1, cloneBucketInfo.getAcls().size());
 
   }
+
+  @Test
+  public void getProtobufMessageEC() {
+    OmBucketInfo omBucketInfo =
+        OmBucketInfo.newBuilder().setBucketName("bucket").setVolumeName("vol1")
+            .setCreationTime(Time.now()).setIsVersionEnabled(false)
+            .setStorageType(StorageType.ARCHIVE).setAcls(Collections
+            .singletonList(new OzoneAcl(IAccessAuthorizer.ACLIdentityType.USER,
+                "defaultUser", IAccessAuthorizer.ACLType.WRITE_ACL,
+                OzoneAcl.AclScope.ACCESS))).build();
+    OzoneManagerProtocolProtos.BucketInfo protobuf = omBucketInfo.getProtobuf();
+    // No EC Config
+    Assert.assertFalse(protobuf.hasDefaultReplicationConfig());
+
+    // Reconstruct object from Proto
+    OmBucketInfo recovered = OmBucketInfo.getFromProtobuf(protobuf);
+    Assert.assertNull(recovered.getDefaultReplicationConfig());
+
+    // EC Config
+    omBucketInfo = OmBucketInfo.newBuilder()
+        .setBucketName("bucket")
+        .setVolumeName("vol1")
+        .setCreationTime(Time.now())
+        .setIsVersionEnabled(false)
+        .setStorageType(StorageType.ARCHIVE)
+        .setAcls(Collections.singletonList(new OzoneAcl(
+            IAccessAuthorizer.ACLIdentityType.USER,
+            "defaultUser", IAccessAuthorizer.ACLType.WRITE_ACL,
+            OzoneAcl.AclScope.ACCESS)))
+        .setDefaultReplicationConfig(
+            new DefaultReplicationConfig(ReplicationType.EC,
+                new ECReplicationConfig(3, 2))).build();
+    protobuf = omBucketInfo.getProtobuf();
+
+    Assert.assertTrue(protobuf.hasDefaultReplicationConfig());
+    Assert.assertEquals(3,
+        protobuf.getDefaultReplicationConfig().getEcReplicationConfig()
+            .getData());
+    Assert.assertEquals(2,
+        protobuf.getDefaultReplicationConfig().getEcReplicationConfig()
+            .getParity());
+
+    // Reconstruct object from Proto
+    recovered = OmBucketInfo.getFromProtobuf(protobuf);
+    Assert.assertEquals(ReplicationType.EC,
+        recovered.getDefaultReplicationConfig().getType());
+    ECReplicationConfig config =
+        recovered.getDefaultReplicationConfig().getEcReplicationConfig();
+    Assert.assertNotNull(config);
+    Assert.assertEquals(3, config.getData());
+    Assert.assertEquals(2, config.getParity());
+  }
 }
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyInfo.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyInfo.java
index 3b04ed4..a6a0ec9 100644
--- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyInfo.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyInfo.java
@@ -18,7 +18,9 @@
 package org.apache.hadoop.ozone.om.helpers;
 
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -26,6 +28,7 @@
 import org.apache.hadoop.ozone.ClientVersion;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo.Builder;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.util.Time;
 import org.junit.Assert;
@@ -36,6 +39,9 @@
 import java.util.Collections;
 import java.util.List;
 
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
 import static org.apache.hadoop.ozone.OzoneAcl.AclScope.ACCESS;
 
 /**
@@ -45,23 +51,67 @@
 
   @Test
   public void protobufConversion() {
-    OmKeyInfo key = new Builder()
+    OmKeyInfo key = createOmKeyInfo(
+        RatisReplicationConfig.getInstance(ReplicationFactor.THREE));
+
+    OmKeyInfo keyAfterSerialization = OmKeyInfo.getFromProtobuf(
+        key.getProtobuf(ClientVersion.CURRENT_VERSION));
+
+    Assert.assertEquals(key, keyAfterSerialization);
+  }
+
+  @Test
+  public void getProtobufMessageEC() {
+    OmKeyInfo key = createOmKeyInfo(
+        RatisReplicationConfig.getInstance(ReplicationFactor.THREE));
+    OzoneManagerProtocolProtos.KeyInfo omKeyProto =
+        key.getProtobuf(ClientVersion.CURRENT_VERSION);
+
+    // No EC Config
+    Assert.assertFalse(omKeyProto.hasEcReplicationConfig());
+    Assert.assertEquals(THREE, omKeyProto.getFactor());
+    Assert.assertEquals(RATIS, omKeyProto.getType());
+
+    // Reconstruct object from Proto
+    OmKeyInfo recovered = OmKeyInfo.getFromProtobuf(omKeyProto);
+    Assert.assertEquals(RATIS,
+        recovered.getReplicationConfig().getReplicationType());
+    Assert.assertTrue(
+        recovered.getReplicationConfig() instanceof RatisReplicationConfig);
+
+    // EC Config
+    key = createOmKeyInfo(new ECReplicationConfig(3, 2));
+    omKeyProto = key.getProtobuf(ClientVersion.CURRENT_VERSION);
+
+    Assert.assertEquals(3, omKeyProto.getEcReplicationConfig().getData());
+    Assert.assertEquals(2, omKeyProto.getEcReplicationConfig().getParity());
+    Assert.assertFalse(omKeyProto.hasFactor());
+    Assert.assertEquals(EC, omKeyProto.getType());
+
+    // Reconstruct object from Proto
+    recovered = OmKeyInfo.getFromProtobuf(omKeyProto);
+    Assert.assertEquals(EC,
+        recovered.getReplicationConfig().getReplicationType());
+    Assert.assertTrue(
+        recovered.getReplicationConfig() instanceof ECReplicationConfig);
+    ECReplicationConfig config =
+        (ECReplicationConfig) recovered.getReplicationConfig();
+    Assert.assertEquals(3, config.getData());
+    Assert.assertEquals(2, config.getParity());
+  }
+
+  private OmKeyInfo createOmKeyInfo(ReplicationConfig replicationConfig) {
+    return new Builder()
         .setKeyName("key1")
         .setBucketName("bucket")
         .setVolumeName("vol1")
         .setCreationTime(123L)
         .setModificationTime(123L)
         .setDataSize(123L)
-        .setReplicationConfig(
-            RatisReplicationConfig.getInstance(ReplicationFactor.THREE))
+        .setReplicationConfig(replicationConfig)
         .addMetadata("key1", "value1")
         .addMetadata("key2", "value2")
         .build();
-
-    OmKeyInfo keyAfterSerialization = OmKeyInfo.getFromProtobuf(
-        key.getProtobuf(ClientVersion.CURRENT_VERSION));
-
-    Assert.assertEquals(key, keyAfterSerialization);
   }
 
   @Test
@@ -161,4 +211,4 @@
         .setPipeline(pipeline)
         .build();
   }
-}
\ No newline at end of file
+}
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmMultipartKeyInfo.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmMultipartKeyInfo.java
index ee850bc..c6f8818 100644
--- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmMultipartKeyInfo.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmMultipartKeyInfo.java
@@ -18,15 +18,23 @@
 
 package org.apache.hadoop.ozone.om.helpers;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
 import org.apache.hadoop.util.Time;
-import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.UUID;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
 
 /**
  * Class to test OmMultipartKeyInfo.
@@ -34,33 +42,97 @@
 public class TestOmMultipartKeyInfo {
 
   @Test
-  public void testCopyObject() {
-    OmMultipartKeyInfo omMultipartKeyInfo = new OmMultipartKeyInfo.Builder()
-        .setUploadID(UUID.randomUUID().toString())
-        .setCreationTime(Time.now())
-        .setReplicationConfig(RatisReplicationConfig
-            .getInstance(HddsProtos.ReplicationFactor.THREE))
+  public void copyObject() {
+    for (ReplicationConfig param : replicationConfigs().collect(toList())) {
+      testCopyObject(param);
+    }
+  }
+
+  //@ParameterizedTest
+  //@MethodSource("replicationConfigs")
+  private void testCopyObject(ReplicationConfig replicationConfig) {
+    // GIVEN
+    OmMultipartKeyInfo subject = createSubject()
+        .setReplicationConfig(replicationConfig)
         .build();
 
-    OmMultipartKeyInfo cloneMultipartKeyInfo = omMultipartKeyInfo.copyObject();
+    // WHEN
+    OmMultipartKeyInfo copy = subject.copyObject();
 
-    Assert.assertEquals(cloneMultipartKeyInfo, omMultipartKeyInfo);
+    // THEN
+    assertNotSame(subject, copy);
+    assertEquals(subject, copy);
+    assertEquals(replicationConfig, copy.getReplicationConfig());
+  }
 
-    // Just setting dummy values for this test.
-    omMultipartKeyInfo.addPartKeyInfo(1,
-        PartKeyInfo.newBuilder().setPartNumber(1).setPartName("/path")
-            .setPartKeyInfo(KeyInfo.newBuilder()
+  @Test
+  public void protoConversion() {
+    for (ReplicationConfig param : replicationConfigs().collect(toList())) {
+      protoConversion(param);
+    }
+  }
+
+  //@ParameterizedTest
+  //@MethodSource("replicationConfigs")
+  private void protoConversion(ReplicationConfig replicationConfig) {
+    // GIVEN
+    OmMultipartKeyInfo subject = createSubject()
+        .setReplicationConfig(replicationConfig)
+        .build();
+
+    // WHEN
+    OzoneManagerProtocolProtos.MultipartKeyInfo proto = subject.getProto();
+    OmMultipartKeyInfo fromProto = OmMultipartKeyInfo.getFromProto(proto);
+
+    // THEN
+    assertEquals(subject, fromProto);
+    assertEquals(replicationConfig, fromProto.getReplicationConfig());
+  }
+
+  private static Stream<ReplicationConfig> replicationConfigs() {
+    return Stream.of(
+        StandaloneReplicationConfig.getInstance(
+            HddsProtos.ReplicationFactor.ONE),
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE),
+        new ECReplicationConfig(3, 2)
+    );
+  }
+
+  @Test
+  public void distinctListOfParts() {
+    // GIVEN
+    OmMultipartKeyInfo subject = createSubject().build();
+    OmMultipartKeyInfo copy = subject.copyObject();
+
+    // WHEN
+    subject.addPartKeyInfo(1, createPart(createKeyInfo()).build());
+
+    // THEN
+    assertEquals(0, copy.getPartKeyInfoMap().size());
+    assertEquals(1, subject.getPartKeyInfoMap().size());
+  }
+
+  private static OmMultipartKeyInfo.Builder createSubject() {
+    return new OmMultipartKeyInfo.Builder()
+        .setUploadID(UUID.randomUUID().toString())
+        .setCreationTime(Time.now());
+  }
+
+  private static PartKeyInfo.Builder createPart(KeyInfo.Builder partKeyInfo) {
+    return PartKeyInfo.newBuilder()
+        .setPartNumber(1)
+        .setPartName("/path")
+        .setPartKeyInfo(partKeyInfo);
+  }
+
+  private static KeyInfo.Builder createKeyInfo() {
+    return KeyInfo.newBuilder()
         .setVolumeName(UUID.randomUUID().toString())
         .setBucketName(UUID.randomUUID().toString())
         .setKeyName(UUID.randomUUID().toString())
-        .setDataSize(100L) // Just set dummy size for testing
+        .setDataSize(100L)
         .setCreationTime(Time.now())
         .setModificationTime(Time.now())
-        .setType(HddsProtos.ReplicationType.RATIS)
-        .setFactor(HddsProtos.ReplicationFactor.ONE).build()).build());
-
-    Assert.assertEquals(0, cloneMultipartKeyInfo.getPartKeyInfoMap().size());
-    Assert.assertEquals(1, omMultipartKeyInfo.getPartKeyInfoMap().size());
-
+        .setType(HddsProtos.ReplicationType.STAND_ALONE);
   }
 }
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestQuotaUtil.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestQuotaUtil.java
new file mode 100644
index 0000000..558676d
--- /dev/null
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestQuotaUtil.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om.helpers;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec.RS;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+
+/**
+ * Tests for the QuotaUtil class.
+ */
+public class TestQuotaUtil {
+
+  private static final int ONE_MB = 1024 * 1024;
+
+  @Test
+  public void testRatisThreeReplication() {
+    ReplicationConfig repConfig = RatisReplicationConfig.getInstance(THREE);
+    long replicatedSize =
+        QuotaUtil.getReplicatedSize(123 * ONE_MB, repConfig);
+    Assert.assertEquals(123 * ONE_MB * 3, replicatedSize);
+  }
+
+  @Test
+  public void testRatisOneReplication() {
+    ReplicationConfig repConfig = RatisReplicationConfig.getInstance(ONE);
+    long replicatedSize =
+        QuotaUtil.getReplicatedSize(123 * ONE_MB, repConfig);
+    Assert.assertEquals(123 * ONE_MB, replicatedSize);
+  }
+
+  @Test
+  public void testECFullStripeReplication() {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+    long dataSize = ONE_MB * 3 * 123; // 123 full stripe
+    long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+    Assert.assertEquals(dataSize + 123 * ONE_MB * 2, replicatedSize);
+  }
+
+  @Test
+  public void testECPartialStripeIntoFirstChunk() {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+    long dataSize = ONE_MB * 3 * 123 + 10; // 123 full stripes, plus 10 bytes
+    long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+    // Expected is 123 parity stripes, plus another 10 bytes in each parity
+    Assert.assertEquals(dataSize + 123 * ONE_MB * 2 + 10 * 2, replicatedSize);
+  }
+
+  @Test
+  public void testECPartialStripeBeyondFirstChunk() {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+    // 123 full stripes, plus 1MB+10 bytes
+    long dataSize = ONE_MB * 3 * 123 + ONE_MB + 10;
+    long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+    // Expected is 123 parity stripes, plus another 1MB in each parity
+    Assert.assertEquals(
+        dataSize + 123 * ONE_MB * 2 + ONE_MB * 2, replicatedSize);
+  }
+
+  @Test
+  public void testECPartialSingleStripeFirstChunk() {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+    long dataSize = 10;
+    long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+    // Expected is 123 parity stripes, plus another 1MB in each parity
+    Assert.assertEquals(dataSize + 10 * 2, replicatedSize);
+  }
+
+  @Test
+  public void testECPartialSingleBeyondFirstChunk() {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+    long dataSize = 2 * ONE_MB + 10;
+    long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+    // Expected is 123 parity stripes, plus another 1MB in each parity
+    Assert.assertEquals(dataSize + ONE_MB * 2, replicatedSize);
+  }
+
+}
diff --git a/hadoop-ozone/dist/src/main/compose/ozone/test.sh b/hadoop-ozone/dist/src/main/compose/ozone/test.sh
index e494470..ac3b273 100755
--- a/hadoop-ozone/dist/src/main/compose/ozone/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/ozone/test.sh
@@ -39,7 +39,7 @@
 
 execute_robot_test scm security/ozone-secure-token.robot
 
-for bucket in link generated; do
+for bucket in erasure link generated; do
   execute_robot_test scm -v BUCKET:${bucket} -N s3-${bucket} s3
 done
 
@@ -57,6 +57,8 @@
 execute_robot_test scm -v SCHEME:ofs -v BUCKET_TYPE:link -N ozonefs-fso-ofs-link ozonefs/ozonefs.robot
 execute_robot_test scm -v SCHEME:o3fs -v BUCKET_TYPE:bucket -N ozonefs-fso-o3fs-bucket ozonefs/ozonefs.robot
 
+execute_robot_test scm ec/basic.robot
+
 stop_docker_env
 
 generate_report
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/docker-compose.yaml
index 24ff9d7..7f52d73 100644
--- a/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/docker-compose.yaml
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/docker-compose.yaml
@@ -124,6 +124,24 @@
       - ${OZONE_VOLUME}/dn3:/data
       - *ozone-dir
       - *transformation
+  dn4:
+    <<: *datanode
+    networks:
+      net:
+        ipv4_address: 10.9.0.18
+    volumes:
+      - ${OZONE_VOLUME}/dn4:/data
+      - *ozone-dir
+      - *transformation
+  dn5:
+    <<: *datanode
+    networks:
+      net:
+        ipv4_address: 10.9.0.19
+    volumes:
+      - ${OZONE_VOLUME}/dn5:/data
+      - *ozone-dir
+      - *transformation
   recon:
     command: ["ozone","recon"]
     <<: *common-config
@@ -131,7 +149,7 @@
       <<: *replication
     networks:
       net:
-        ipv4_address: 10.9.0.18
+        ipv4_address: 10.9.0.20
     ports:
       - 9888:9888
     volumes:
@@ -145,7 +163,7 @@
       <<: *replication
     networks:
       net:
-        ipv4_address: 10.9.0.19
+        ipv4_address: 10.9.0.21
     ports:
       - 9878:9878
     volumes:
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/load.sh b/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/load.sh
index 902eb12..43c6688 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/load.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/load.sh
@@ -25,4 +25,4 @@
 
 export COMPOSE_FILE="$TEST_DIR/compose/ha/docker-compose.yaml"
 export OM_SERVICE_ID=omservice
-create_data_dirs "${OZONE_VOLUME}"/{om1,om2,om3,dn1,dn2,dn3,recon,s3g,scm}
+create_data_dirs "${OZONE_VOLUME}"/{om1,om2,om3,dn1,dn2,dn3,dn4,dn5,recon,s3g,scm}
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/docker-compose.yaml
index 4b1d64e..14617cc 100644
--- a/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/docker-compose.yaml
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/docker-compose.yaml
@@ -101,6 +101,24 @@
       - ${OZONE_VOLUME}/dn3:/data
       - *ozone-dir
       - *transformation
+  dn4:
+    <<: *datanode
+    networks:
+      net:
+        ipv4_address: 10.9.0.16
+    volumes:
+      - ${OZONE_VOLUME}/dn4:/data
+      - *ozone-dir
+      - *transformation
+  dn5:
+    <<: *datanode
+    networks:
+      net:
+        ipv4_address: 10.9.0.17
+    volumes:
+      - ${OZONE_VOLUME}/dn5:/data
+      - *ozone-dir
+      - *transformation
   recon:
     command: ["ozone","recon"]
     <<: *common-config
@@ -108,7 +126,7 @@
       <<: *replication
     networks:
       net:
-        ipv4_address: 10.9.0.16
+        ipv4_address: 10.9.0.18
     ports:
       - 9888:9888
     volumes:
@@ -122,7 +140,7 @@
       <<: *replication
     networks:
       net:
-        ipv4_address: 10.9.0.17
+        ipv4_address: 10.9.0.19
     ports:
       - 9878:9878
     volumes:
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/load.sh b/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/load.sh
index a8980e7..0e8fc05 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/load.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/load.sh
@@ -24,4 +24,4 @@
 source "$TEST_DIR/testlib.sh"
 
 export COMPOSE_FILE="$TEST_DIR/compose/non-ha/docker-compose.yaml"
-create_data_dirs "${OZONE_VOLUME}"/{om,dn1,dn2,dn3,recon,s3g,scm}
+create_data_dirs "${OZONE_VOLUME}"/{om,dn1,dn2,dn3,dn4,dn5,recon,s3g,scm}
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/test.sh b/hadoop-ozone/dist/src/main/compose/upgrade/test.sh
index 687f9bd..35358e3 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/test.sh
@@ -16,7 +16,7 @@
 # limitations under the License.
 
 # Version that will be run using the local build.
-: "${OZONE_CURRENT_VERSION:=1.2.0}"
+: "${OZONE_CURRENT_VERSION:=1.3.0}"
 export OZONE_CURRENT_VERSION
 
 TEST_DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null && pwd )
@@ -31,8 +31,10 @@
 
 RESULT_DIR="$ALL_RESULT_DIR" create_results_dir
 
-# Upgrade tests to be run.
-run_test non-rolling-upgrade 1.1.0 1.2.0
+# Upgrade tests to be run. In CI we want to run just one set, but for a release
+# we might advise the release manager to run the full matrix.
+#run_test non-rolling-upgrade 1.1.0 1.3.0
+run_test non-rolling-upgrade 1.2.1 1.3.0
 
 generate_report "upgrade" "$ALL_RESULT_DIR"
 
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/testlib.sh b/hadoop-ozone/dist/src/main/compose/upgrade/testlib.sh
index 1b6b05d..b3fa1df 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/testlib.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/testlib.sh
@@ -150,3 +150,11 @@
 check_scm_mlv() {
   check_mlv "$1" /data/metadata/scm/current/VERSION "$2"
 }
+
+check_ec_is_disabled() {
+  execute_robot_test scm --include pre-finalized-ec-tests ec/upgrade-ec-check.robot
+}
+
+check_ec_is_enabled() {
+  execute_robot_test scm --include post-finalized-ec-tests ec/upgrade-ec-check.robot
+}
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
new file mode 100755
index 0000000..f414ddf
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
@@ -0,0 +1,84 @@
+#!/usr/bin/env bash
+# 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.
+
+source "$TEST_DIR"/testlib.sh
+
+# Helper function, not a callback.
+_check_hdds_mlvs() {
+  mlv="$1"
+  check_scm_mlv scm "$mlv"
+  check_dn_mlv dn1 "$mlv"
+  check_dn_mlv dn2 "$mlv"
+  check_dn_mlv dn3 "$mlv"
+}
+
+# Helper function, not a callback.
+_check_om_mlvs() {
+  mlv="$1"
+  check_om_mlv om1 "$mlv"
+  check_om_mlv om2 "$mlv"
+  check_om_mlv om3 "$mlv"
+}
+
+setup() {
+  export OZONE_OM_PREPARE='true'
+}
+
+with_old_version() {
+  generate old1
+  validate old1
+}
+
+with_new_version_pre_finalized() {
+  _check_hdds_mlvs 2
+  _check_om_mlvs 0
+
+  validate old1
+#   HDDS-6261: overwrite the same keys intentionally
+  generate old1 --exclude create-volume-and-bucket
+
+  generate new1
+  validate new1
+
+  check_ec_is_disabled
+}
+
+with_old_version_downgraded() {
+  validate old1
+  validate new1
+
+  generate old2
+  validate old2
+
+  # HDDS-6261: overwrite the same keys again to trigger the precondition check
+  # that exists <= 1.1.0 OM
+  generate old1 --exclude create-volume-and-bucket
+}
+
+with_new_version_finalized() {
+  _check_hdds_mlvs 3
+  _check_om_mlvs 1
+
+  validate old1
+  validate new1
+  validate old2
+
+  generate new2
+  validate new2
+
+  check_ec_is_enabled
+}
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/driver.sh b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/driver.sh
index e59bec5..7b95238 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/driver.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/driver.sh
@@ -72,7 +72,6 @@
 OUTPUT_NAME="$OZONE_UPGRADE_TO"-pre-finalized
 OZONE_KEEP_RESULTS=true start_docker_env
 callback with_new_version_pre_finalized
-
 prepare_oms
 stop_docker_env
 prepare_for_image "$OZONE_UPGRADE_FROM"
diff --git a/hadoop-ozone/dist/src/main/compose/xcompat/docker-config b/hadoop-ozone/dist/src/main/compose/xcompat/docker-config
index 8bbb4d1..29b3696 100644
--- a/hadoop-ozone/dist/src/main/compose/xcompat/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/xcompat/docker-config
@@ -14,6 +14,8 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+CORE-SITE.XML_fs.ofs.impl=org.apache.hadoop.fs.ozone.RootedOzoneFileSystem
+
 OZONE-SITE.XML_hdds.datanode.dir=/data/hdds
 OZONE-SITE.XML_hdds.scm.safemode.min.datanode=3
 OZONE-SITE.XML_ozone.metadata.dirs=/data/metadata
diff --git a/hadoop-ozone/dist/src/main/compose/xcompat/test.sh b/hadoop-ozone/dist/src/main/compose/xcompat/test.sh
index a46590f..f583a2b 100755
--- a/hadoop-ozone/dist/src/main/compose/xcompat/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/xcompat/test.sh
@@ -73,6 +73,45 @@
   KEEP_RUNNING=false stop_docker_env
 }
 
+test_ec_cross_compatibility() {
+  echo "Running Erasure Coded storage backward compatibility tests."
+  local cluster_versions_with_ec="1.3.0"
+  local non_ec_client_versions="1.0.0 1.1.0 1.2.1"
+
+  for cluster_version in ${cluster_versions_with_ec}; do
+    export COMPOSE_FILE=new-cluster.yaml:clients.yaml cluster_version=${cluster_version}
+    OZONE_KEEP_RESULTS=true start_docker_env 5
+
+    echo -n "Generating data locally...   "
+    dd if=/dev/urandom of=/tmp/1mb bs=1048576 count=1 >/dev/null 2>&1
+    dd if=/dev/urandom of=/tmp/2mb bs=1048576 count=2 >/dev/null 2>&1
+    dd if=/dev/urandom of=/tmp/3mb bs=1048576 count=3 >/dev/null 2>&1
+    echo "done"
+    echo -n "Copy data into client containers...   "
+    for container in $(docker ps --format '{{.Names}}' | grep client); do
+      docker cp /tmp/1mb ${container}:/tmp/1mb
+      docker cp /tmp/2mb ${container}:/tmp/2mb
+      docker cp /tmp/3mb ${container}:/tmp/3mb
+    done
+    echo "done"
+    rm -f /tmp/1mb /tmp/2mb /tmp/3mb
+
+
+    local prefix=$(LC_CTYPE=C tr -dc '[:alnum:]' < /dev/urandom | head -c 5 | tr '[:upper:]' '[:lower:]')
+    OZONE_DIR=/opt/hadoop
+    execute_robot_test new_client --include setup-ec-data -N "xcompat-cluster-${cluster_version}-setup-data" -v prefix:"${prefix}" ec/backward-compat.robot
+     OZONE_DIR=/opt/ozone
+
+    for client_version in ${non_ec_client_versions}; do
+      client="old_client_${client_version//./_}"
+      unset OUTPUT_PATH
+      execute_robot_test "${client}" --include test-ec-compat -N "xcompat-cluster-${cluster_version}-client-${client_version}-read-${cluster_version}" -v prefix:"${prefix}" ec/backward-compat.robot
+    done
+
+    KEEP_RUNNING=false stop_docker_env
+  done
+}
+
 create_results_dir
 
 # current cluster with various clients
@@ -84,4 +123,6 @@
   COMPOSE_FILE=old-cluster.yaml:clients.yaml test_cross_compatibility ${cluster_version}
 done
 
+test_ec_cross_compatibility
+
 generate_report
diff --git a/hadoop-ozone/dist/src/main/license/jar-report.txt b/hadoop-ozone/dist/src/main/license/jar-report.txt
index da8350d..3911c4d 100644
--- a/hadoop-ozone/dist/src/main/license/jar-report.txt
+++ b/hadoop-ozone/dist/src/main/license/jar-report.txt
@@ -64,6 +64,7 @@
 share/ozone/lib/hdds-config.jar
 share/ozone/lib/hdds-container-service.jar
 share/ozone/lib/hdds-docs.jar
+share/ozone/lib/hdds-erasurecode.jar
 share/ozone/lib/hdds-hadoop-dependency-client.jar
 share/ozone/lib/hdds-hadoop-dependency-server.jar
 share/ozone/lib/hdds-interface-admin.jar
diff --git a/hadoop-ozone/dist/src/main/org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory b/hadoop-ozone/dist/src/main/org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory
new file mode 100644
index 0000000..23b1a14
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory
@@ -0,0 +1,18 @@
+#
+#   Licensed 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.
+#
+org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory
+org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawErasureCoderFactory
+org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory
+org.apache.hadoop.io.erasurecode.rawcoder.RSLegacyRawErasureCoderFactory
+org.apache.hadoop.io.erasurecode.rawcoder.XORRawErasureCoderFactory
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot b/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
index 5752eed..f0d8fa8 100644
--- a/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
@@ -54,11 +54,11 @@
                         Should Not contain   ${output}   OPEN
 
 List containers with replication factor ONE
-    ${output} =         Execute          ozone admin container list --factor=ONE
+    ${output} =         Execute          ozone admin container list -t RATIS -r ONE
                         Should Not contain   ${output}   THREE
 
 List containers with replication factor THREE
-    ${output} =         Execute          ozone admin container list --factor=THREE
+    ${output} =         Execute          ozone admin container list -t RATIS -r THREE
                         Should Not contain   ${output}   ONE
 
 Container info
diff --git a/hadoop-ozone/dist/src/main/smoketest/ec/backward-compat.robot b/hadoop-ozone/dist/src/main/smoketest/ec/backward-compat.robot
new file mode 100644
index 0000000..74ab547
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ec/backward-compat.robot
@@ -0,0 +1,103 @@
+# 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.
+
+*** Settings ***
+Documentation       Test EC backward compatibility
+Library             OperatingSystem
+Resource            lib.resource
+
+*** Test Cases ***
+Setup Cluster Data
+    [Tags]  setup-ec-data
+    Prepare Data For Xcompat Tests
+
+Test Read Key Compat
+    [Tags]  test-ec-compat
+    Key Should Match Local File     /${prefix}vol1/${prefix}ratis/${prefix}3mb      /tmp/3mb
+    Key Should Match Local File     /${prefix}vol1/${prefix}default/${prefix}3mb    /tmp/3mb
+
+    ${result} =     Execute and checkrc         ozone sh key get /${prefix}vol1/${prefix}ec/${prefix}3mb /tmp/${prefix}3mb       255
+                    Should Contain  ${result}   NOT_SUPPORTED_OPERATION
+
+Test Listing Compat
+    [Tags]  test-ec-compat
+    ${result} =     Execute     ozone sh volume list | jq -r '.name'
+                    Should contain  ${result}   ${prefix}vol1
+    ${result} =     Execute     ozone sh bucket list /${prefix}vol1/ | jq -r '.name'
+                    Should contain  ${result}   ${prefix}default
+                    Should contain  ${result}   ${prefix}ratis
+                    Should contain  ${result}   ${prefix}ec
+    ${result} =     Execute     ozone sh key list /${prefix}vol1/${prefix}default/ | jq -r '[.name, .replicationType, (.replicationFactor | tostring)] | join (" ")'
+                    Should contain  ${result}   ${prefix}3mb RATIS 3
+    ${result} =     Execute     ozone sh key list /${prefix}vol1/${prefix}ratis/ | jq -r '[.name, .replicationType, (.replicationFactor | tostring)] | join (" ")'
+                    Should contain  ${result}   ${prefix}3mb RATIS 3
+
+    ${result} =     Execute and checkrc         ozone sh key list /${prefix}vol1/${prefix}ec/   255
+                    Should contain  ${result}   NOT_SUPPORTED_OPERATION
+
+Test Info Compat
+    [Tags]  test-ec-compat
+    ${result} =     Execute     ozone sh volume info ${prefix}vol1 | jq -r '.name'
+                    Should contain  ${result}   ${prefix}vol1
+    ${result} =     Execute     ozone sh bucket info /${prefix}vol1/${prefix}default | jq -r '[.name, .replicationType, .replicationFactor] | join (" ")'
+                    Should contain  ${result}   ${prefix}default        # there is no replication config in the old client for bucket info
+    ${result} =     Execute     ozone sh bucket info /${prefix}vol1/${prefix}ratis | jq -r '[.name, .replicationType, .replicationFactor] | join (" ")'
+                    Should contain  ${result}   ${prefix}ratis        # there is no replication config in the old client for bucket info
+    ${result} =     Execute     ozone sh bucket info /${prefix}vol1/${prefix}ec | jq -r '[.name, .replicationType, .replicationFactor] | join (" ")'
+                    Should contain  ${result}   ${prefix}ec        # there is no replication config in the old client for bucket info
+
+Test FS Compat
+    [Tags]  test-ec-compat
+    ${result} =     Execute     ozone fs -ls ofs://om/
+                    Should contain  ${result}   /${prefix}vol1
+    ${result} =     Execute     ozone fs -ls ofs://om/${prefix}vol1/
+                    Should contain  ${result}   /${prefix}vol1/${prefix}default
+                    Should contain  ${result}   /${prefix}vol1/${prefix}ratis
+                    Should contain  ${result}   /${prefix}vol1/${prefix}ec
+    ${result} =     Execute     ozone fs -ls ofs://om/${prefix}vol1/${prefix}default/${prefix}3mb
+                    Should contain  ${result}   /${prefix}vol1/${prefix}default/${prefix}3mb
+    ${result} =     Execute     ozone fs -ls ofs://om/${prefix}vol1/${prefix}ratis/${prefix}3mb
+                    Should contain  ${result}   /${prefix}vol1/${prefix}ratis/${prefix}3mb
+
+    ${result} =     Execute and checkrc    ozone fs -ls ofs://om/${prefix}vol1/${prefix}ec/     1
+                    Should contain  ${result}   ls: The list of keys contains keys with Erasure Coded replication set
+    ${result} =     Execute and checkrc    ozone fs -ls ofs://om/${prefix}vol1/${prefix}ec/${prefix}3mb     1
+                    Should contain  ${result}   : No such file or directory
+    ${result} =     Execute and checkrc    ozone fs -get ofs://om/${prefix}vol1/${prefix}ec/${prefix}3mb    1
+                    Should contain  ${result}   : No such file or directory
+
+Test FS Client Can Read Own Writes
+    [Tags]  test-ec-compat
+    Execute         ozone fs -put /tmp/1mb ofs://om/${prefix}vol1/${prefix}default/${prefix}1mb
+    Execute         ozone fs -put /tmp/1mb ofs://om/${prefix}vol1/${prefix}ratis/${prefix}1mb
+    Execute         ozone fs -put /tmp/1mb ofs://om/${prefix}vol1/${prefix}ec/${prefix}1mb
+    Key Should Match Local File     /${prefix}vol1/${prefix}ratis/${prefix}1mb      /tmp/1mb
+    Key Should Match Local File     /${prefix}vol1/${prefix}ratis/${prefix}1mb      /tmp/1mb
+    Key Should Match Local File     /${prefix}vol1/${prefix}ratis/${prefix}1mb      /tmp/1mb
+    Execute         ozone fs -rm -skipTrash ofs://om/${prefix}vol1/${prefix}default/${prefix}1mb
+    Execute         ozone fs -rm -skipTrash ofs://om/${prefix}vol1/${prefix}ratis/${prefix}1mb
+    Execute         ozone fs -rm -skipTrash ofs://om/${prefix}vol1/${prefix}ec/${prefix}1mb
+
+Test Client Can Read Own Writes
+    [Tags]  test-ec-compat
+    Execute         ozone sh key put /${prefix}vol1/${prefix}default/${prefix}2mb /tmp/2mb
+    Execute         ozone sh key put /${prefix}vol1/${prefix}ratis/${prefix}2mb /tmp/2mb
+    Execute         ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}2mb /tmp/2mb
+    Key Should Match Local File     /${prefix}vol1/${prefix}ratis/${prefix}2mb      /tmp/2mb
+    Key Should Match Local File     /${prefix}vol1/${prefix}ratis/${prefix}2mb      /tmp/2mb
+    Key Should Match Local File     /${prefix}vol1/${prefix}ratis/${prefix}2mb      /tmp/2mb
+    Execute         ozone sh key delete /${prefix}vol1/${prefix}default/${prefix}2mb
+    Execute         ozone sh key delete /${prefix}vol1/${prefix}ratis/${prefix}2mb
+    Execute         ozone sh key delete /${prefix}vol1/${prefix}ec/${prefix}2mb
diff --git a/hadoop-ozone/dist/src/main/smoketest/ec/basic.robot b/hadoop-ozone/dist/src/main/smoketest/ec/basic.robot
new file mode 100644
index 0000000..751211d
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ec/basic.robot
@@ -0,0 +1,64 @@
+# 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.
+
+*** Settings ***
+Documentation       Test EC shell commands
+Library             OperatingSystem
+Resource            ../commonlib.robot
+Resource            ../ozone-lib/shell.robot
+Resource            lib.resource
+Suite Setup         Prepare For Tests
+
+*** Test Cases ***
+Test Bucket Creation
+    ${result} =     Execute             ozone sh volume create /${prefix}vol1
+                    Should not contain  ${result}       Failed
+    ${result} =     Execute             ozone sh bucket create /${prefix}vol1/${prefix}default
+                    Should not contain  ${result}       Failed
+                    Verify Bucket Empty Replication Config      /${prefix}vol1/${prefix}default
+    ${result} =     Execute             ozone sh bucket create --replication 3 --type RATIS /${prefix}vol1/${prefix}ratis
+                    Should not contain  ${result}       Failed
+                    Verify Bucket Replica Replication Config    /${prefix}vol1/${prefix}ratis   RATIS   THREE
+    ${result} =     Execute             ozone sh bucket create --replication rs-3-2-1024k --type EC /${prefix}vol1/${prefix}ec
+                    Should not contain  ${result}       Failed
+                    Verify Bucket EC Replication Config    /${prefix}vol1/${prefix}ec    RS    3    2    1048576
+
+Test Key Creation EC Bucket
+                    Execute                             ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}1mb /tmp/1mb
+                    Execute                             ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}2mb /tmp/2mb
+                    Execute                             ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}3mb /tmp/3mb
+                    Execute                             ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}100mb /tmp/100mb
+
+                    Key Should Match Local File         /${prefix}vol1/${prefix}ec/${prefix}1mb      /tmp/1mb
+                    Key Should Match Local File         /${prefix}vol1/${prefix}ec/${prefix}2mb      /tmp/2mb
+                    Key Should Match Local File         /${prefix}vol1/${prefix}ec/${prefix}3mb      /tmp/3mb
+                    Key Should Match Local File         /${prefix}vol1/${prefix}ec/${prefix}100mb    /tmp/100mb
+
+                    Verify Key EC Replication Config    /${prefix}vol1/${prefix}ec/${prefix}1mb    RS    3    2    1048576
+
+Test Key Creation Default Bucket
+                    Execute                             ozone sh key put /${prefix}vol1/${prefix}default/${prefix}1mb /tmp/1mb
+                    Key Should Match Local File         /${prefix}vol1/${prefix}default/${prefix}1mb      /tmp/1mb
+                    Verify Key Replica Replication Config   /${prefix}vol1/${prefix}default/${prefix}1mb     RATIS    THREE
+
+Test Ratis Key EC Bucket
+                    Execute                       ozone sh key put --replication=THREE --type=RATIS /${prefix}vol1/${prefix}ec/${prefix}1mbRatis /tmp/1mb
+                    Key Should Match Local File   /${prefix}vol1/${prefix}ec/${prefix}1mbRatis    /tmp/1mb
+                    Verify Key Replica Replication Config   /${prefix}vol1/${prefix}ec/${prefix}1mbRatis    RATIS   THREE
+
+Test EC Key Ratis Bucket
+                    Execute                             ozone sh key put --replication=rs-3-2-1024k --type=EC /${prefix}vol1/${prefix}ratis/${prefix}1mbEC /tmp/1mb
+                    Key Should Match Local File         /${prefix}vol1/${prefix}ratis/${prefix}1mbEC    /tmp/1mb
+                    Verify Key EC Replication Config    /${prefix}vol1/${prefix}ratis/${prefix}1mbEC    RS    3    2    1048576
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/smoketest/ec/lib.resource b/hadoop-ozone/dist/src/main/smoketest/ec/lib.resource
new file mode 100644
index 0000000..7c774a2
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ec/lib.resource
@@ -0,0 +1,42 @@
+# 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.
+
+*** Settings ***
+Documentation       Keywords and other resources used by EC tests
+Library             OperatingSystem
+Resource            ../commonlib.robot
+Resource            ../ozone-lib/shell.robot
+
+*** Variables ***
+${SCM}       scm
+
+*** Keywords ***
+Prepare For Tests
+    ${random}    Generate Random String  5  [NUMBERS]
+    Set Suite Variable  ${prefix}  ${random}
+    Execute             dd if=/dev/urandom of=/tmp/1mb bs=1048576 count=1
+    Execute             dd if=/dev/urandom of=/tmp/2mb bs=1048576 count=2
+    Execute             dd if=/dev/urandom of=/tmp/3mb bs=1048576 count=3
+    Execute             dd if=/dev/urandom of=/tmp/100mb bs=1048576 count=100
+
+# xcompat/test.sh creates unified test data files in /tmp for client containers
+Prepare Data For Xcompat Tests
+    Execute             ozone sh volume create /${prefix}vol1
+    Execute             ozone sh bucket create /${prefix}vol1/${prefix}default
+    Execute             ozone sh bucket create --replication 3 --type RATIS /${prefix}vol1/${prefix}ratis
+    Execute             ozone sh bucket create --replication rs-3-2-1024k --type EC /${prefix}vol1/${prefix}ec
+    Execute             ozone sh key put /${prefix}vol1/${prefix}default/${prefix}3mb /tmp/3mb
+    Execute             ozone sh key put /${prefix}vol1/${prefix}ratis/${prefix}3mb /tmp/3mb
+    Execute             ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}3mb /tmp/3mb
diff --git a/hadoop-ozone/dist/src/main/smoketest/ec/upgrade-ec-check.robot b/hadoop-ozone/dist/src/main/smoketest/ec/upgrade-ec-check.robot
new file mode 100644
index 0000000..dbfd9e8
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ec/upgrade-ec-check.robot
@@ -0,0 +1,45 @@
+# 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.
+
+*** Settings ***
+Documentation       Test EC during upgrade
+Library             OperatingSystem
+Resource            lib.resource
+
+*** Test Cases ***
+Test EC Prior To Finalization
+    [Tags]  pre-finalized-ec-tests
+    Execute         ozone sh volume create /ectest
+    ${result} =     Execute and checkrc     ozone sh bucket create --replication rs-3-2-1024k --type EC /ectest/ectest     255
+                    Should contain  ${result}   NOT_SUPPORTED_OPERATION
+    Execute         ozone sh bucket create /ectest/testpropchange
+    ${result} =     Execute and checkrc     ozone sh bucket set-replication-config -r rs-3-2-1024k -t EC /ectest/testpropchange     255
+                    Should contain  ${result}   NOT_SUPPORTED_OPERATION
+    ${result} =     Execute and checkrc     ozone sh key put -r rs-3-2-1024k -t EC /ectest/testpropchange/core-site.xml /etc/hadoop/core-site.xml     255
+                    Should contain  ${result}   NOT_SUPPORTED_OPERATION
+
+
+
+Test EC After Finalization
+    [Tags]  post-finalized-ec-tests
+    Execute         ozone sh volume create /ectest-new
+    Execute         ozone sh bucket create --replication rs-3-2-1024k --type EC /ectest-new/ectest
+                    Verify Bucket EC Replication Config     /ectest-new/ectest  RS  3   2   1048576
+    Execute         ozone sh bucket create /ectest-new/testpropchange
+    Execute         ozone sh bucket set-replication-config -r rs-3-2-1024k -t EC /ectest-new/testpropchange
+                    Verify Bucket EC Replication Config     /ectest-new/testpropchange  RS  3   2   1048576
+    Execute         ozone sh key put -r rs-3-2-1024k -t EC /ectest-new/ectest/core-site.xml /etc/hadoop/core-site.xml
+                    Key Should Match Local File     /ectest-new/ectest/core-site.xml        /etc/hadoop/core-site.xml
+                    Verify Key EC Replication Config    /ectest-new/ectest/core-site.xml    RS  3   2   1048576
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot
index 23d305e..07753a3 100644
--- a/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot
@@ -61,3 +61,41 @@
     ${output} =    Execute          ozone sh key put ${key} ${file}
                    Should not contain  ${output}       Failed
     Log            Uploaded ${file} to ${key}
+
+Verify Bucket Empty Replication Config
+    [arguments]    ${bucket}
+    ${result} =    Execute                      ozone sh bucket info ${bucket} | jq -r '.replicationConfig'
+                   Should Be Equal          ${result}       null
+
+Verify Bucket Replica Replication Config
+    [arguments]    ${bucket}    ${type}    ${factor}
+    ${result} =    Execute                      ozone sh bucket info ${bucket} | jq -r '.replicationConfig.replicationType, .replicationConfig.replicationFactor'
+                   Verify Replica Replication Config    ${result}   ${type}     ${factor}
+
+Verify Key Replica Replication Config
+    [arguments]    ${key}    ${type}    ${factor}
+    ${result} =    Execute                      ozone sh key info ${key} | jq -r '.replicationConfig.replicationType, .replicationConfig.replicationFactor'
+                   Verify Replica Replication Config    ${result}   ${type}     ${factor}
+
+Verify Replica Replication Config
+    [arguments]    ${result}    ${type}    ${factor}
+                   Should Match Regexp      ${result}       ^(?m)${type}$
+                   Should Match Regexp      ${result}       ^(?m)${factor}$
+
+Verify Bucket EC Replication Config
+    [arguments]    ${bucket}    ${encoding}    ${data}    ${parity}    ${chunksize}
+    ${result} =    Execute                      ozone sh bucket info ${bucket} | jq -r '.replicationConfig.replicationType, .replicationConfig.codec, .replicationConfig.data, .replicationConfig.parity, .replicationConfig.ecChunkSize'
+                   Verify EC Replication Config     ${result}    ${encoding}    ${data}    ${parity}    ${chunksize}
+
+Verify Key EC Replication Config
+    [arguments]    ${key}    ${encoding}    ${data}    ${parity}    ${chunksize}
+    ${result} =    Execute                      ozone sh key info ${key} | jq -r '.replicationConfig.replicationType, .replicationConfig.codec, .replicationConfig.data, .replicationConfig.parity, .replicationConfig.ecChunkSize'
+                   Verify EC Replication Config     ${result}    ${encoding}    ${data}    ${parity}    ${chunksize}
+
+Verify EC Replication Config
+    [arguments]    ${result}    ${encoding}    ${data}    ${parity}    ${chunksize}
+                   Should Match Regexp      ${result}       ^(?m)EC$
+                   Should Match Regexp      ${result}       ^(?m)${encoding}$
+                   Should Match Regexp      ${result}       ^(?m)${data}$
+                   Should Match Regexp      ${result}       ^(?m)${parity}$
+                   Should Match Regexp      ${result}       ^(?m)${chunksize}$
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot b/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot
index f888d22..5e03d0c 100644
--- a/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot
@@ -45,8 +45,8 @@
                    Execute               ozone fs -copyFromLocal NOTICE.txt ${DEEP_URL}/
     ${result} =    Execute               ozone sh key list ${VOLUME}/${BUCKET} | jq -r '.[].name'
                    Should contain        ${result}         NOTICE.txt
-    ${result} =    Execute               ozone sh key info ${VOLUME}/${BUCKET}/${DEEP_DIR}/NOTICE.txt | jq -r '.replicationFactor'
-                   Should Be Equal       ${result}         3
+    ${result} =    Execute               ozone sh key info ${VOLUME}/${BUCKET}/${DEEP_DIR}/NOTICE.txt | jq -r '.replicationConfig.replicationFactor'
+                   Should Be Equal       ${result}         THREE
 
 Put
                    Execute               ozone fs -put NOTICE.txt ${DEEP_URL}/PUTFILE.txt
diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot b/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot
index 96939c9..6f3bbef 100644
--- a/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot
@@ -105,6 +105,7 @@
                        Set Suite Variable                        ${BUCKET}
                        Run Keyword if                            '${BUCKET}' == 'link'                 Setup links for S3 tests
                        Run Keyword if                            '${BUCKET}' == 'encrypted'            Create encrypted bucket
+                       Run Keyword if                            '${BUCKET}' == 'erasure'              Create EC bucket
 
 Setup links for S3 tests
     ${exists} =        Bucket Exists    o3://${OM_SERVICE_ID}/s3v/link
@@ -124,6 +125,11 @@
     Execute           ozone sh bucket link o3://${OM_SERVICE_ID}/legacy/source-bucket o3://${OM_SERVICE_ID}/s3v/${bucket}
     [return]          ${bucket}
 
+Create EC bucket
+    ${exists} =        Bucket Exists    o3://${OM_SERVICE_ID}/s3v/erasure
+    Return From Keyword If    ${exists}
+    Execute            ozone sh bucket create --replication rs-3-2-1024k --type EC o3://${OM_SERVICE_ID}/s3v/erasure
+
 Generate random prefix
     ${random} =          Generate Ozone String
                          Set Suite Variable  ${PREFIX}  ${random}
diff --git a/hadoop-ozone/dist/src/main/smoketest/upgrade/prepare.robot b/hadoop-ozone/dist/src/main/smoketest/upgrade/prepare.robot
index 0f6d7a0..b1b4095 100644
--- a/hadoop-ozone/dist/src/main/smoketest/upgrade/prepare.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/upgrade/prepare.robot
@@ -19,7 +19,7 @@
 Test Timeout        5 minutes
 Test Setup          Run Keyword if    '${SECURITY_ENABLED}' == 'true'    Kinit test user     testuser     testuser.keytab
 
-** Test Cases ***
+*** Test Cases ***
 Prepare Ozone Manager
     ${result} =        Execute      ozone admin om prepare -id %{OM_SERVICE_ID}
                        Wait Until Keyword Succeeds      3min       10sec     Should contain   ${result}   OM Preparation successful!
diff --git a/hadoop-ozone/integration-test/pom.xml b/hadoop-ozone/integration-test/pom.xml
index c3bce81..cea5163 100644
--- a/hadoop-ozone/integration-test/pom.xml
+++ b/hadoop-ozone/integration-test/pom.xml
@@ -110,6 +110,11 @@
     </dependency>
     <dependency>
       <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-params</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
       <artifactId>junit-jupiter-migrationsupport</artifactId>
       <scope>test</scope>
     </dependency>
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
index bf24ae9..4c4bdc6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
@@ -21,6 +21,8 @@
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -29,6 +31,7 @@
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -42,8 +45,12 @@
 import org.apache.hadoop.fs.TrashPolicy;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -82,6 +89,7 @@
 import static org.junit.Assert.fail;
 
 import org.apache.ozone.test.LambdaTestUtils;
+import org.apache.ozone.test.TestClock;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -166,7 +174,7 @@
     conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
         bucketLayout.name());
     cluster = MiniOzoneCluster.newBuilder(conf)
-            .setNumDatanodes(3)
+            .setNumDatanodes(5)
             .build();
     cluster.waitForClusterToBeReady();
 
@@ -1188,6 +1196,66 @@
   }
 
   @Test
+  public void testCreateKeyShouldUseRefreshedBucketReplicationConfig()
+      throws IOException {
+    OzoneBucket bucket =
+        TestDataUtil.createVolumeAndBucket(cluster, bucketLayout);
+    final TestClock testClock = new TestClock(Instant.now(), ZoneOffset.UTC);
+
+    String rootPath = String
+        .format("%s://%s.%s/", OzoneConsts.OZONE_URI_SCHEME, bucket.getName(),
+            bucket.getVolumeName());
+
+    // Set the fs.defaultFS and start the filesystem
+    Configuration conf = new OzoneConfiguration(cluster.getConf());
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath);
+    // Set the number of keys to be processed during batch operate.
+    OzoneFileSystem o3FS = (OzoneFileSystem) FileSystem.get(conf);
+
+    //Let's reset the clock to control the time.
+    ((BasicOzoneClientAdapterImpl) (o3FS.getAdapter())).setClock(testClock);
+
+    createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key"),
+        ReplicationType.RATIS);
+
+    bucket.setReplicationConfig(new ECReplicationConfig("rs-3-2-1024k"));
+
+    //After changing the bucket policy, it should create ec key, but o3fs will
+    // refresh after some time. So, it will be sill old type.
+    createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key1"),
+        ReplicationType.RATIS);
+
+    testClock.fastForward(300 * 1000 + 1);
+
+    //After client bucket refresh time, it should create new type what is
+    // available on bucket at that moment.
+    createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key2"),
+        ReplicationType.EC);
+
+    // Rechecking the same steps with changing to Ratis again to check the
+    // behavior is consistent.
+    bucket.setReplicationConfig(
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
+
+    createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key3"),
+        ReplicationType.EC);
+
+    testClock.fastForward(300 * 1000 + 1);
+
+    createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key4"),
+        ReplicationType.RATIS);
+  }
+
+  private void createKeyAndAssertKeyType(OzoneBucket bucket,
+      OzoneFileSystem o3FS, Path keyPath, ReplicationType expectedType)
+      throws IOException {
+    o3FS.createFile(keyPath).build().close();
+    Assert.assertEquals(expectedType.name(),
+        bucket.getKey(o3FS.pathToKey(keyPath)).getReplicationConfig()
+            .getReplicationType().name());
+  }
+
+  @Test
   public void testGetTrashRoots() throws IOException {
     String username = UserGroupInformation.getCurrentUser().getShortUserName();
     Path trashRoot = new Path(OZONE_URI_DELIMITER, TRASH_PREFIX);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java
index 5c5c0e1..b88da8a 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java
@@ -31,7 +31,12 @@
 import org.apache.hadoop.fs.TrashPolicy;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OFSPath;
 import org.apache.hadoop.ozone.OzoneAcl;
@@ -81,6 +86,7 @@
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.UUID;
 import java.util.concurrent.TimeoutException;
 import java.util.stream.Collectors;
 
@@ -214,7 +220,7 @@
     conf.set(OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS,
         OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS_NATIVE);
     cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(3)
+        .setNumDatanodes(5)
         .build();
     cluster.waitForClusterToBeReady();
     objectStore = cluster.getClient().getObjectStore();
@@ -1575,6 +1581,66 @@
   }
 
   @Test
+  public void testBucketDefaultsShouldNotBeInheritedToFileForNonEC()
+      throws Exception {
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setStorageType(StorageType.DISK);
+    builder.setBucketLayout(BucketLayout.LEGACY);
+    builder.setDefaultReplicationConfig(
+        new DefaultReplicationConfig(ReplicationType.STAND_ALONE,
+            ReplicationFactor.ONE));
+    BucketArgs omBucketArgs = builder.build();
+    String vol = UUID.randomUUID().toString();
+    String buck = UUID.randomUUID().toString();
+    final OzoneBucket bucket100 = TestDataUtil
+        .createVolumeAndBucket(cluster, vol, buck, BucketLayout.LEGACY,
+            omBucketArgs);
+    Assert.assertEquals(ReplicationType.STAND_ALONE.name(),
+        bucket100.getReplicationConfig().getReplicationType().name());
+
+    // Bucket has default STAND_ALONE and client has default RATIS.
+    // In this case, it should not inherit from bucket
+    try (OzoneFSOutputStream file = adapter
+        .createFile(vol + "/" + buck + "/test", (short) 3, true, false)) {
+      file.write(new byte[1024]);
+    }
+    OFSPath ofsPath = new OFSPath(vol + "/" + buck + "/test");
+    final OzoneBucket bucket = adapter.getBucket(ofsPath, false);
+    final OzoneKeyDetails key = bucket.getKey(ofsPath.getKeyName());
+    Assert.assertEquals(key.getReplicationConfig().getReplicationType().name(),
+        ReplicationType.RATIS.name());
+  }
+
+  @Test
+  public void testBucketDefaultsShouldBeInheritedToFileForEC()
+      throws Exception {
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setStorageType(StorageType.DISK);
+    builder.setBucketLayout(BucketLayout.LEGACY);
+    builder.setDefaultReplicationConfig(
+        new DefaultReplicationConfig(ReplicationType.EC,
+            new ECReplicationConfig("RS-3-2-1024")));
+    BucketArgs omBucketArgs = builder.build();
+    String vol = UUID.randomUUID().toString();
+    String buck = UUID.randomUUID().toString();
+    final OzoneBucket bucket101 = TestDataUtil
+        .createVolumeAndBucket(cluster, vol, buck, BucketLayout.LEGACY,
+            omBucketArgs);
+    Assert.assertEquals(ReplicationType.EC.name(),
+        bucket101.getReplicationConfig().getReplicationType().name());
+    // Bucket has default EC and client has default RATIS.
+    // In this case, it should inherit from bucket
+    try (OzoneFSOutputStream file = adapter
+        .createFile(vol + "/" + buck + "/test", (short) 3, true, false)) {
+      file.write(new byte[1024]);
+    }
+    OFSPath ofsPath = new OFSPath(vol + "/" + buck + "/test");
+    final OzoneBucket bucket = adapter.getBucket(ofsPath, false);
+    final OzoneKeyDetails key = bucket.getKey(ofsPath.getKeyName());
+    Assert.assertEquals(ReplicationType.EC.name(),
+        key.getReplicationConfig().getReplicationType().name());
+  }
+
   public void testNonPrivilegedUserMkdirCreateBucket() throws IOException {
     // This test is only meaningful when ACL is enabled
     Assume.assumeTrue("ACL is not enabled. Skipping this test as it requires " +
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
index ac0ad91..6920dd4 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
@@ -48,6 +48,8 @@
 import org.apache.hadoop.hdds.scm.ha.SCMHANodeDetails;
 import org.apache.hadoop.hdds.scm.ha.SCMHAUtils;
 import org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.proxy.SCMClientConfig;
@@ -376,16 +378,21 @@
     startRecon();
   }
 
-  private void waitForHddsDatanodesStop() throws TimeoutException,
-      InterruptedException {
+  private void waitForHddsDatanodeToStop(DatanodeDetails dn)
+      throws TimeoutException, InterruptedException {
     GenericTestUtils.waitFor(() -> {
-      final int healthy = scm.getNodeCount(HEALTHY);
-      boolean isReady = healthy == hddsDatanodes.size();
-      if (!isReady) {
-        LOG.info("Waiting on {} datanodes out of {} to be marked unhealthy.",
-            healthy, hddsDatanodes.size());
+      NodeStatus status;
+      try {
+        status = scm.getScmNodeManager().getNodeStatus(dn);
+      } catch (NodeNotFoundException e) {
+        return true;
       }
-      return isReady;
+      if (status.equals(NodeStatus.inServiceHealthy())) {
+        LOG.info("Waiting on datanode to be marked stale.");
+        return false;
+      } else {
+        return true;
+      }
     }, 1000, waitForClusterToBeReadyTimeout);
   }
 
@@ -410,7 +417,7 @@
     hddsDatanodes.remove(i);
     if (waitForDatanode) {
       // wait for node to be removed from SCM healthy node list.
-      waitForHddsDatanodesStop();
+      waitForHddsDatanodeToStop(datanodeService.getDatanodeDetails());
     }
     String[] args = new String[] {};
     HddsDatanodeService service =
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestDataUtil.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestDataUtil.java
index 2983270..4b1e724 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestDataUtil.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestDataUtil.java
@@ -56,6 +56,19 @@
   public static OzoneBucket createVolumeAndBucket(MiniOzoneCluster cluster,
       String volumeName, String bucketName, BucketLayout bucketLayout)
       throws IOException {
+    BucketArgs omBucketArgs;
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setStorageType(StorageType.DISK);
+    builder.setBucketLayout(bucketLayout);
+    omBucketArgs = builder.build();
+
+    return createVolumeAndBucket(cluster, volumeName, bucketName, bucketLayout,
+        omBucketArgs);
+  }
+
+  public static OzoneBucket createVolumeAndBucket(MiniOzoneCluster cluster,
+      String volumeName, String bucketName, BucketLayout bucketLayout,
+      BucketArgs omBucketArgs) throws IOException {
     String userName = "user" + RandomStringUtils.randomNumeric(5);
     String adminName = "admin" + RandomStringUtils.randomNumeric(5);
 
@@ -69,11 +82,6 @@
     objectStore.createVolume(volumeName, volumeArgs);
 
     OzoneVolume volume = objectStore.getVolume(volumeName);
-    BucketArgs omBucketArgs;
-    BucketArgs.Builder builder = BucketArgs.newBuilder();
-    builder.setStorageType(StorageType.DISK);
-    builder.setBucketLayout(bucketLayout);
-    omBucketArgs = builder.build();
 
     volume.createBucket(bucketName, omBucketArgs);
     return volume.getBucket(bucketName);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
index ef561e6..b239621 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
@@ -62,11 +62,17 @@
     xmlPropsToSkipCompare.add("ozone.om.decommissioned.nodes" +
         ".EXAMPLEOMSERVICEID");
     xmlPropsToSkipCompare.add("ozone.scm.nodes.EXAMPLESCMSERVICEID");
+    xmlPropsToSkipCompare.add("ozone.scm.nodes.EXAMPLESCMSERVICEID");
     xmlPrefixToSkipCompare.add("ipc.client.rpc-timeout.ms");
     xmlPropsToSkipCompare.add("ozone.om.leader.election.minimum.timeout" +
         ".duration"); // Deprecated config
     configurationPropsToSkipCompare
         .add(ScmConfig.ConfigStrings.HDDS_SCM_INIT_DEFAULT_LAYOUT_VERSION);
+    // Currently replication and type configs moved to server side.
+    configurationPropsToSkipCompare
+        .add(OzoneConfigKeys.OZONE_REPLICATION);
+    configurationPropsToSkipCompare
+        .add(OzoneConfigKeys.OZONE_REPLICATION_TYPE);
     configurationPropsToSkipCompare
         .add(OzoneConfigKeys.OZONE_CLIENT_REQUIRED_OM_VERSION_MIN_KEY);
     configurationPropsToSkipCompare
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestECKeyOutputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestECKeyOutputStream.java
new file mode 100644
index 0000000..948c481
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestECKeyOutputStream.java
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.client.rpc;
+
+import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
+import org.apache.hadoop.ozone.client.io.KeyOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.container.TestHelper;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
+
+/**
+ * Tests key output stream.
+ */
+public class TestECKeyOutputStream {
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf = new OzoneConfiguration();
+  private static OzoneClient client;
+  private static ObjectStore objectStore;
+  private static int chunkSize;
+  private static int flushSize;
+  private static int maxFlushSize;
+  private static int blockSize;
+  private static String volumeName;
+  private static String bucketName;
+  private static String keyString;
+  private static int dataBlocks = 3;
+  private static int inputSize = dataBlocks * chunkSize;
+  private static byte[][] inputChunks = new byte[dataBlocks][chunkSize];
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   */
+  @BeforeClass
+  public static void init() throws Exception {
+    chunkSize = 1024;
+    flushSize = 2 * chunkSize;
+    maxFlushSize = 2 * flushSize;
+    blockSize = 2 * maxFlushSize;
+
+    OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumType(ContainerProtos.ChecksumType.NONE);
+    clientConfig.setStreamBufferFlushDelay(false);
+    conf.setFromObject(clientConfig);
+
+    conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
+    // If SCM detects dead node too quickly, then container would be moved to
+    // closed state and all in progress writes will get exception. To avoid
+    // that, we are just keeping higher timeout and none of the tests depending
+    // on deadnode detection timeout currently.
+    conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 30, TimeUnit.SECONDS);
+    conf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, 60, TimeUnit.SECONDS);
+    conf.setTimeDuration("hdds.ratis.raft.server.rpc.slowness.timeout", 300,
+        TimeUnit.SECONDS);
+    conf.setTimeDuration(
+        "hdds.ratis.raft.server.notification.no-leader.timeout", 300,
+        TimeUnit.SECONDS);
+    conf.setQuietMode(false);
+    conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 4,
+        StorageUnit.MB);
+    conf.setTimeDuration(HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL, 500,
+        TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL, 1,
+        TimeUnit.SECONDS);
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(10)
+        .setTotalPipelineNumLimit(10).setBlockSize(blockSize)
+        .setChunkSize(chunkSize).setStreamBufferFlushSize(flushSize)
+        .setStreamBufferMaxSize(maxFlushSize)
+        .setStreamBufferSizeUnit(StorageUnit.BYTES).build();
+    cluster.waitForClusterToBeReady();
+    client = OzoneClientFactory.getRpcClient(conf);
+    objectStore = client.getObjectStore();
+    keyString = UUID.randomUUID().toString();
+    volumeName = "testeckeyoutputstream";
+    bucketName = volumeName;
+    objectStore.createVolume(volumeName);
+    objectStore.getVolume(volumeName).createBucket(bucketName);
+    initInputChunks();
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testCreateKeyWithECReplicationConfig() throws Exception {
+    try (OzoneOutputStream key = TestHelper
+        .createKey(keyString, new ECReplicationConfig(3, 2,
+              ECReplicationConfig.EcCodec.RS, chunkSize), inputSize,
+            objectStore, volumeName, bucketName)) {
+      Assert.assertTrue(key.getOutputStream() instanceof ECKeyOutputStream);
+    }
+  }
+
+  @Test
+  public void testCreateKeyWithOutBucketDefaults() throws Exception {
+    OzoneVolume volume = objectStore.getVolume(volumeName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+    try (OzoneOutputStream out = bucket.createKey("myKey", inputSize)) {
+      Assert.assertTrue(out.getOutputStream() instanceof KeyOutputStream);
+      for (int i = 0; i < inputChunks.length; i++) {
+        out.write(inputChunks[i]);
+      }
+    }
+  }
+
+  @Test
+  public void testCreateKeyWithBucketDefaults() throws Exception {
+    String myBucket = UUID.randomUUID().toString();
+    OzoneVolume volume = objectStore.getVolume(volumeName);
+    final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
+    bucketArgs.setDefaultReplicationConfig(
+        new DefaultReplicationConfig(ReplicationType.EC,
+            new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+                chunkSize)));
+
+    volume.createBucket(myBucket, bucketArgs.build());
+    OzoneBucket bucket = volume.getBucket(myBucket);
+
+    try (OzoneOutputStream out = bucket.createKey(keyString, inputSize)) {
+      Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+      for (int i = 0; i < inputChunks.length; i++) {
+        out.write(inputChunks[i]);
+      }
+    }
+    byte[] buf = new byte[chunkSize];
+    try (OzoneInputStream in = bucket.readKey(keyString)) {
+      for (int i = 0; i < inputChunks.length; i++) {
+        int read = in.read(buf, 0, chunkSize);
+        Assert.assertEquals(chunkSize, read);
+        Assert.assertTrue(Arrays.equals(buf, inputChunks[i]));
+      }
+    }
+  }
+
+  @Test
+  public void testOverwriteECKeyWithRatisKey() throws Exception {
+    String myBucket = UUID.randomUUID().toString();
+    OzoneVolume volume = objectStore.getVolume(volumeName);
+    final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
+    volume.createBucket(myBucket, bucketArgs.build());
+    OzoneBucket bucket = volume.getBucket(myBucket);
+    createKeyAndCheckReplicationConfig(keyString, bucket,
+        new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+            chunkSize));
+
+    //Overwrite with RATIS/THREE
+    createKeyAndCheckReplicationConfig(keyString, bucket,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
+
+    //Overwrite with RATIS/ONE
+    createKeyAndCheckReplicationConfig(keyString, bucket,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE));
+  }
+
+  @Test
+  public void testOverwriteRatisKeyWithECKey() throws Exception {
+    String myBucket = UUID.randomUUID().toString();
+    OzoneVolume volume = objectStore.getVolume(volumeName);
+    final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
+    volume.createBucket(myBucket, bucketArgs.build());
+    OzoneBucket bucket = volume.getBucket(myBucket);
+
+    createKeyAndCheckReplicationConfig(keyString, bucket,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
+    // Overwrite with EC key
+    createKeyAndCheckReplicationConfig(keyString, bucket,
+        new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+            chunkSize));
+  }
+
+  private void createKeyAndCheckReplicationConfig(String keyName,
+      OzoneBucket bucket, ReplicationConfig replicationConfig)
+      throws IOException {
+    try (OzoneOutputStream out = bucket
+        .createKey(keyName, inputSize, replicationConfig, new HashMap<>())) {
+      for (int i = 0; i < inputChunks.length; i++) {
+        out.write(inputChunks[i]);
+      }
+    }
+    OzoneKeyDetails key = bucket.getKey(keyName);
+    Assert.assertEquals(replicationConfig, key.getReplicationConfig());
+  }
+
+  @Test
+  public void testCreateRatisKeyAndWithECBucketDefaults() throws Exception {
+    OzoneBucket bucket = getOzoneBucket();
+    try (OzoneOutputStream out = bucket.createKey(
+        "testCreateRatisKeyAndWithECBucketDefaults", 2000,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE),
+        new HashMap<>())) {
+      Assert.assertTrue(out.getOutputStream() instanceof KeyOutputStream);
+      for (int i = 0; i < inputChunks.length; i++) {
+        out.write(inputChunks[i]);
+      }
+    }
+  }
+
+  @Test
+  public void test13ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(13);
+  }
+
+  @Test
+  public void test15ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(15);
+  }
+
+  @Test
+  public void test20ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(20);
+  }
+
+  @Test
+  public void test21ChunksInSingleWriteOp() throws IOException {
+    testMultipleChunksInSingleWriteOp(21);
+  }
+
+  public void testMultipleChunksInSingleWriteOp(int numChunks)
+      throws IOException {
+    byte[] inputData = getInputBytes(numChunks);
+    final OzoneBucket bucket = getOzoneBucket();
+    String keyName = "testMultipleChunksInSingleWriteOp" + numChunks;
+    try (OzoneOutputStream out = bucket.createKey(keyName, 4096,
+        new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+            chunkSize), new HashMap<>())) {
+      out.write(inputData);
+    }
+
+    validateContent(inputData, bucket, bucket.getKey(keyName));
+  }
+
+  @Test
+  public void testECContainerKeysCount()
+      throws IOException, InterruptedException, TimeoutException {
+    byte[] inputData = getInputBytes(1);
+    final OzoneBucket bucket = getOzoneBucket();
+    ContainerOperationClient containerOperationClient =
+        new ContainerOperationClient(conf);
+
+    ECReplicationConfig repConfig = new ECReplicationConfig(
+        3, 2, ECReplicationConfig.EcCodec.RS, chunkSize);
+    // Close all EC pipelines so we must get a fresh pipeline and hence
+    // container for this test.
+    PipelineManager pm =
+        cluster.getStorageContainerManager().getPipelineManager();
+    for (Pipeline p : pm.getPipelines(repConfig)) {
+      pm.closePipeline(p, true);
+    }
+
+    String keyName = UUID.randomUUID().toString();
+    try (OzoneOutputStream out = bucket.createKey(keyName, 4096,
+        repConfig, new HashMap<>())) {
+      out.write(inputData);
+    }
+    OzoneKeyDetails key = bucket.getKey(keyName);
+    long currentKeyContainerID =
+        key.getOzoneKeyLocations().get(0).getContainerID();
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return containerOperationClient.getContainer(currentKeyContainerID)
+            .getNumberOfKeys() == 1;
+      } catch (IOException exception) {
+        Assert.fail("Unexpected exception " + exception);
+        return false;
+      }
+    }, 100, 10000);
+    validateContent(inputData, bucket, key);
+  }
+
+  private void validateContent(byte[] inputData, OzoneBucket bucket,
+      OzoneKey key) throws IOException {
+    try (OzoneInputStream is = bucket.readKey(key.getName())) {
+      byte[] fileContent = new byte[inputData.length];
+      Assert.assertEquals(inputData.length, is.read(fileContent));
+      Assert.assertEquals(new String(inputData, UTF_8),
+          new String(fileContent, UTF_8));
+    }
+  }
+
+  private OzoneBucket getOzoneBucket() throws IOException {
+    String myBucket = UUID.randomUUID().toString();
+    OzoneVolume volume = objectStore.getVolume(volumeName);
+    final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
+    bucketArgs.setDefaultReplicationConfig(
+        new DefaultReplicationConfig(ReplicationType.EC,
+            new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+                chunkSize)));
+
+    volume.createBucket(myBucket, bucketArgs.build());
+    return volume.getBucket(myBucket);
+  }
+
+  private static void initInputChunks() {
+    for (int i = 0; i < dataBlocks; i++) {
+      inputChunks[i] = getBytesWith(i + 1, chunkSize);
+    }
+  }
+
+  private static byte[] getBytesWith(int singleDigitNumber, int total) {
+    StringBuilder builder = new StringBuilder(singleDigitNumber);
+    for (int i = 1; i <= total; i++) {
+      builder.append(singleDigitNumber);
+    }
+    return builder.toString().getBytes(UTF_8);
+  }
+
+  @Test
+  public void testWriteShouldSucceedWhenDNKilled() throws Exception {
+    int numChunks = 3;
+    byte[] inputData = getInputBytes(numChunks);
+    final OzoneBucket bucket = getOzoneBucket();
+    String keyName = "testWriteShouldSucceedWhenDNKilled" + numChunks;
+    DatanodeDetails nodeToKill = null;
+    try {
+      try (OzoneOutputStream out = bucket.createKey(keyName, 1024,
+          new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+              chunkSize), new HashMap<>())) {
+        out.write(inputData);
+        // Kill a node from first pipeline
+        nodeToKill =
+            ((ECKeyOutputStream) out.getOutputStream()).getStreamEntries()
+                .get(0).getPipeline().getFirstNode();
+        cluster.shutdownHddsDatanode(nodeToKill);
+
+        out.write(inputData);
+        // Check the second blockGroup pipeline to make sure that the failed not
+        // is not selected.
+        Assert.assertFalse(
+            ((ECKeyOutputStream) out.getOutputStream()).getStreamEntries()
+                .get(1).getPipeline().getNodes().contains(nodeToKill));
+      }
+
+      try (OzoneInputStream is = bucket.readKey(keyName)) {
+        // We wrote "inputData" twice, so do two reads and ensure the correct
+        // data comes back.
+        for (int i = 0; i < 2; i++) {
+          byte[] fileContent = new byte[inputData.length];
+          Assert.assertEquals(inputData.length, is.read(fileContent));
+          Assert.assertEquals(new String(inputData, UTF_8),
+              new String(fileContent, UTF_8));
+        }
+      }
+    } finally {
+      cluster.restartHddsDatanode(nodeToKill, true);
+    }
+  }
+
+  private byte[] getInputBytes(int numChunks) {
+    byte[] inputData = new byte[numChunks * chunkSize];
+    for (int i = 0; i < numChunks; i++) {
+      int start = (i * chunkSize);
+      Arrays.fill(inputData, start, start + chunkSize - 1,
+          String.valueOf(i % 9).getBytes(UTF_8)[0]);
+    }
+    return inputData;
+  }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java
index 9ea04d4..293725f 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java
@@ -18,9 +18,11 @@
 package org.apache.hadoop.ozone.client.rpc;
 
 import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneTestUtils;
@@ -620,8 +622,10 @@
     OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
         bucket.listParts(keyName, uploadID, 0, 3);
 
-    Assert.assertEquals(RATIS,
-        ozoneMultipartUploadPartListParts.getReplicationType());
+    Assert.assertEquals(
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+        ozoneMultipartUploadPartListParts.getReplicationConfig());
+
     Assert.assertEquals(3,
         ozoneMultipartUploadPartListParts.getPartInfoList().size());
 
@@ -722,8 +726,9 @@
     OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
         bucket.listParts(keyName, uploadID, 0, 2);
 
-    Assert.assertEquals(RATIS,
-        ozoneMultipartUploadPartListParts.getReplicationType());
+    Assert.assertEquals(
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+        ozoneMultipartUploadPartListParts.getReplicationConfig());
 
     Assert.assertEquals(2,
         ozoneMultipartUploadPartListParts.getPartInfoList().size());
@@ -821,8 +826,9 @@
 
     Assert.assertEquals(0,
         ozoneMultipartUploadPartListParts.getPartInfoList().size());
-    Assert.assertEquals(RATIS,
-        ozoneMultipartUploadPartListParts.getReplicationType());
+    Assert.assertEquals(
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+        ozoneMultipartUploadPartListParts.getReplicationConfig());
 
     // As we don't have any parts with greater than partNumberMarker and list
     // is not truncated, so it should return false here.
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
index 5bdd344..527b3bb 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.ozone.client.rpc;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
index 6272afa..fb50b81 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
@@ -35,12 +35,17 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
 
 import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.StorageType;
@@ -98,6 +103,7 @@
 import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.QuotaUtil;
 import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
@@ -146,6 +152,8 @@
 import org.junit.jupiter.api.MethodOrderer;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestMethodOrder;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 /**
  * This is an abstract class to test all the public facing APIs of Ozone
@@ -188,7 +196,7 @@
     //  for testZReadKeyWithUnhealthyContainerReplica.
     conf.set("ozone.scm.stale.node.interval", "10s");
     cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(3)
+        .setNumDatanodes(5)
         .setTotalPipelineNumLimit(10)
         .setScmId(scmId)
         .setClusterId(clusterId)
@@ -589,7 +597,7 @@
 
   @Test
   public void testCreateBucketWithVersioning()
-      throws IOException, OzoneClientException {
+      throws IOException {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
     store.createVolume(volumeName);
@@ -637,6 +645,23 @@
   }
 
   @Test
+  public void testCreateBucketWithReplicationConfig()
+      throws IOException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    ReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    BucketArgs bucketArgs = BucketArgs.newBuilder()
+        .setDefaultReplicationConfig(new DefaultReplicationConfig(repConfig))
+        .build();
+    volume.createBucket(bucketName, bucketArgs);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+    Assert.assertEquals(bucketName, bucket.getName());
+    Assert.assertEquals(repConfig, bucket.getReplicationConfig());
+  }
+
+  @Test
   public void testCreateBucketWithAllArgument()
       throws IOException {
     String volumeName = UUID.randomUUID().toString();
@@ -645,18 +670,21 @@
         ACLType.ALL, ACCESS);
     List<OzoneAcl> acls = new ArrayList<>();
     acls.add(userAcl);
+    ReplicationConfig repConfig = new ECReplicationConfig(3, 2);
     store.createVolume(volumeName);
     OzoneVolume volume = store.getVolume(volumeName);
     BucketArgs.Builder builder = BucketArgs.newBuilder();
     builder.setVersioning(true)
         .setStorageType(StorageType.SSD)
-        .setAcls(acls);
+        .setAcls(acls)
+        .setDefaultReplicationConfig(new DefaultReplicationConfig(repConfig));
     volume.createBucket(bucketName, builder.build());
     OzoneBucket bucket = volume.getBucket(bucketName);
     Assert.assertEquals(bucketName, bucket.getName());
     Assert.assertEquals(true, bucket.getVersioning());
     Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
     Assert.assertTrue(bucket.getAcls().contains(userAcl));
+    Assert.assertEquals(repConfig, bucket.getReplicationConfig());
   }
 
   @Test
@@ -670,7 +698,6 @@
         "Bucket or Volume name has an unsupported" +
             " character : #",
         () -> volume.createBucket(bucketName));
-
   }
 
   @Test
@@ -799,7 +826,6 @@
     Assert.assertEquals(StorageType.SSD, newBucket.getStorageType());
   }
 
-
   @Test
   public void testDeleteBucket()
       throws Exception {
@@ -818,8 +844,8 @@
     );
   }
 
-  private boolean verifyRatisReplication(String volumeName, String bucketName,
-      String keyName, ReplicationType type, ReplicationFactor factor)
+  private void verifyReplication(String volumeName, String bucketName,
+      String keyName, ReplicationConfig replication)
       throws IOException {
     OmKeyArgs keyArgs = new OmKeyArgs.Builder()
         .setVolumeName(volumeName)
@@ -827,22 +853,13 @@
         .setKeyName(keyName)
         .setRefreshPipeline(true)
         .build();
-    HddsProtos.ReplicationType replicationType =
-        HddsProtos.ReplicationType.valueOf(type.toString());
-    HddsProtos.ReplicationFactor replicationFactor =
-        HddsProtos.ReplicationFactor.valueOf(factor.getValue());
     OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs);
     for (OmKeyLocationInfo info:
         keyInfo.getLatestVersionLocations().getLocationList()) {
       ContainerInfo container =
           storageContainerLocationClient.getContainer(info.getContainerID());
-      if (!ReplicationConfig.getLegacyFactor(container.getReplicationConfig())
-          .equals(replicationFactor) || (
-          container.getReplicationType() != replicationType)) {
-        return false;
-      }
+      Assert.assertEquals(replication, container.getReplicationConfig());
     }
-    return true;
   }
 
   @Test
@@ -870,9 +887,9 @@
       OzoneInputStream is = bucket.readKey(keyName);
       byte[] fileContent = new byte[value.getBytes(UTF_8).length];
       is.read(fileContent);
-      Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
-          keyName, RATIS,
-          ONE));
+      verifyReplication(volumeName, bucketName, keyName,
+          RatisReplicationConfig.getInstance(
+              HddsProtos.ReplicationFactor.ONE));
       Assert.assertEquals(value, new String(fileContent, UTF_8));
       Assert.assertFalse(key.getCreationTime().isBefore(testStartTime));
       Assert.assertFalse(key.getModificationTime().isBefore(testStartTime));
@@ -1012,6 +1029,64 @@
         store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
   }
 
+  // TODO: testBucketQuota overlaps with testBucketUsedBytes,
+  //       do cleanup when EC branch gets merged into master.
+  @ParameterizedTest
+  @MethodSource("replicationConfigs")
+  void testBucketQuota(ReplicationConfig repConfig) throws IOException {
+    int blockSize = (int) ozoneManager.getConfiguration().getStorageSize(
+        OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
+
+    for (int i = 0; i <= repConfig.getRequiredNodes(); i++) {
+      bucketQuotaTestHelper(i * blockSize, repConfig);
+      bucketQuotaTestHelper(i * blockSize + 1, repConfig);
+    }
+  }
+
+  private void bucketQuotaTestHelper(int keyLength, ReplicationConfig repConfig)
+      throws IOException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+    long blockSize = (long) ozoneManager.getConfiguration().getStorageSize(
+        OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    byte[] value = new byte[keyLength];
+    int dataGroupSize = repConfig instanceof ECReplicationConfig ?
+        ((ECReplicationConfig) repConfig).getData() : 1;
+    long preAllocatedBlocks = Math.min(ozoneManager.getPreallocateBlocksMax(),
+        (keyLength - 1) / (blockSize * dataGroupSize) + 1);
+    long preAllocatedSpace = preAllocatedBlocks * blockSize
+        * repConfig.getRequiredNodes();
+    long keyQuota = QuotaUtil.getReplicatedSize(keyLength, repConfig);
+
+    OzoneOutputStream out = bucket.createKey(keyName, keyLength,
+        repConfig, new HashMap<>());
+    Assert.assertEquals(preAllocatedSpace,
+        store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+    out.write(value);
+    out.close();
+    Assert.assertEquals(keyQuota,
+        store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+
+    out = bucket.createKey(keyName, keyLength, repConfig, new HashMap<>());
+    Assert.assertEquals(keyQuota + preAllocatedSpace,
+        store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+    out.write(value);
+    out.close();
+    Assert.assertEquals(keyQuota,
+        store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+
+    bucket.deleteKey(keyName);
+    Assert.assertEquals(0L,
+        store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+  }
+
   @Test
   public void testVolumeUsedNamespace() throws IOException {
     String volumeName = UUID.randomUUID().toString();
@@ -1144,8 +1219,10 @@
     OzoneVolume volume = store.getVolume(volumeName);
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
+    ReplicationConfig replication = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.ONE);
     OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
-        RATIS, ONE);
+        replication);
 
     assertNotNull(multipartInfo);
     String uploadID = multipartInfo.getUploadID();
@@ -1228,8 +1305,9 @@
       byte[] fileContent = new byte[value.getBytes(UTF_8).length];
       is.read(fileContent);
       is.close();
-      Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
-          keyName, ReplicationType.RATIS, ONE));
+      verifyReplication(volumeName, bucketName, keyName,
+          RatisReplicationConfig.getInstance(
+              HddsProtos.ReplicationFactor.ONE));
       Assert.assertEquals(value, new String(fileContent, UTF_8));
       Assert.assertFalse(key.getCreationTime().isBefore(testStartTime));
       Assert.assertFalse(key.getModificationTime().isBefore(testStartTime));
@@ -1262,9 +1340,9 @@
       byte[] fileContent = new byte[value.getBytes(UTF_8).length];
       is.read(fileContent);
       is.close();
-      Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
-          keyName, ReplicationType.RATIS,
-          THREE));
+      verifyReplication(volumeName, bucketName, keyName,
+          RatisReplicationConfig.getInstance(
+              HddsProtos.ReplicationFactor.THREE));
       Assert.assertEquals(value, new String(fileContent, UTF_8));
       Assert.assertFalse(key.getCreationTime().isBefore(testStartTime));
       Assert.assertFalse(key.getModificationTime().isBefore(testStartTime));
@@ -1303,9 +1381,9 @@
           byte[] fileContent = new byte[data.getBytes(UTF_8).length];
           is.read(fileContent);
           is.close();
-          Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
-              keyName, ReplicationType.RATIS,
-              THREE));
+          verifyReplication(volumeName, bucketName, keyName,
+              RatisReplicationConfig.getInstance(
+                  HddsProtos.ReplicationFactor.THREE));
           Assert.assertEquals(data, new String(fileContent, UTF_8));
           Assert.assertFalse(key.getCreationTime().isBefore(testStartTime));
           Assert.assertFalse(key.getModificationTime().isBefore(testStartTime));
@@ -2034,7 +2112,6 @@
           .startsWith(bucketBaseNameA + i + "-"));
     }
     Assert.assertFalse(volBBucketAIter.hasNext());
-
   }
 
   @Test
@@ -2050,6 +2127,43 @@
   }
 
   @Test
+  public void testListBucketsReplicationConfig()
+      throws Exception {
+    String volumeName = UUID.randomUUID().toString();
+    getStore().createVolume(volumeName);
+    OzoneVolume volume = getStore().getVolume(volumeName);
+
+    // bucket-level replication config: null (default)
+    String bucketName = UUID.randomUUID().toString();
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.listBuckets(bucketName).next();
+    Assert.assertNull(bucket.getReplicationConfig());
+
+    // bucket-level replication config: EC/rs-3-2-1024k
+    String ecBucketName = UUID.randomUUID().toString();
+    ReplicationConfig ecRepConfig = new ECReplicationConfig(3, 2);
+    BucketArgs ecBucketArgs = BucketArgs.newBuilder()
+        .setDefaultReplicationConfig(
+            new DefaultReplicationConfig(ecRepConfig))
+        .build();
+    volume.createBucket(ecBucketName, ecBucketArgs);
+    OzoneBucket ecBucket = volume.listBuckets(ecBucketName).next();
+    Assert.assertEquals(ecRepConfig, ecBucket.getReplicationConfig());
+
+    // bucket-level replication config: RATIS/THREE
+    String ratisBucketName = UUID.randomUUID().toString();
+    ReplicationConfig ratisRepConfig = ReplicationConfig
+        .fromTypeAndFactor(RATIS, THREE);
+    BucketArgs ratisBucketArgs = BucketArgs.newBuilder()
+        .setDefaultReplicationConfig(
+            new DefaultReplicationConfig(ratisRepConfig))
+        .build();
+    volume.createBucket(ratisBucketName, ratisBucketArgs);
+    OzoneBucket ratisBucket = volume.listBuckets(ratisBucketName).next();
+    Assert.assertEquals(ratisRepConfig, ratisBucket.getReplicationConfig());
+  }
+
+  @Test
   public void testListKey()
       throws IOException {
     String volumeA = "vol-a-" + RandomStringUtils.randomNumeric(5);
@@ -2199,9 +2313,18 @@
     }
   }
 
-  @Test
-  public void testInitiateMultipartUploadWithReplicationInformationSet() throws
-      IOException {
+  static Stream<ReplicationConfig> replicationConfigs() {
+    return Stream.of(
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE),
+        new ECReplicationConfig(3, 2)
+    );
+  }
+
+  @ParameterizedTest
+  @MethodSource("replicationConfigs")
+  void testInitiateMultipartUpload(ReplicationConfig replicationConfig)
+      throws IOException {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
     String keyName = UUID.randomUUID().toString();
@@ -2211,7 +2334,7 @@
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
     OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
-        RATIS, ONE);
+        replicationConfig);
 
     assertNotNull(multipartInfo);
     String uploadID = multipartInfo.getUploadID();
@@ -2223,7 +2346,7 @@
     // Call initiate multipart upload for the same key again, this should
     // generate a new uploadID.
     multipartInfo = bucket.initiateMultipartUpload(keyName,
-        RATIS, ONE);
+        replicationConfig);
 
     assertNotNull(multipartInfo);
     Assert.assertEquals(volumeName, multipartInfo.getVolumeName());
@@ -2266,8 +2389,10 @@
     assertNotNull(multipartInfo.getUploadID());
   }
 
-  @Test
-  public void testUploadPartWithNoOverride() throws IOException {
+  @ParameterizedTest
+  @MethodSource("replicationConfigs")
+  void testUploadPartWithNoOverride(ReplicationConfig replication)
+      throws IOException {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
     String keyName = UUID.randomUUID().toString();
@@ -2278,7 +2403,7 @@
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
     OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
-        RATIS, ONE);
+        replication);
 
     assertNotNull(multipartInfo);
     String uploadID = multipartInfo.getUploadID();
@@ -2296,13 +2421,13 @@
         .getCommitUploadPartInfo();
 
     assertNotNull(commitUploadPartInfo);
-    String partName = commitUploadPartInfo.getPartName();
     assertNotNull(commitUploadPartInfo.getPartName());
-
   }
 
-  @Test
-  public void testUploadPartOverrideWithStandAlone() throws IOException {
+  @ParameterizedTest
+  @MethodSource("replicationConfigs")
+  void testUploadPartOverride(ReplicationConfig replication)
+      throws IOException {
 
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
@@ -2315,7 +2440,7 @@
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
     OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
-        RATIS, ONE);
+        replication);
 
     assertNotNull(multipartInfo);
     String uploadID = multipartInfo.getUploadID();
@@ -2362,66 +2487,6 @@
   }
 
   @Test
-  public void testUploadPartOverrideWithRatis() throws IOException {
-
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    String keyName = UUID.randomUUID().toString();
-    String sampleData = "sample Value";
-
-    store.createVolume(volumeName);
-    OzoneVolume volume = store.getVolume(volumeName);
-    volume.createBucket(bucketName);
-    OzoneBucket bucket = volume.getBucket(bucketName);
-    OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
-        ReplicationType.RATIS, THREE);
-
-    assertNotNull(multipartInfo);
-    String uploadID = multipartInfo.getUploadID();
-    Assert.assertEquals(volumeName, multipartInfo.getVolumeName());
-    Assert.assertEquals(bucketName, multipartInfo.getBucketName());
-    Assert.assertEquals(keyName, multipartInfo.getKeyName());
-    assertNotNull(multipartInfo.getUploadID());
-
-    int partNumber = 1;
-
-    OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName,
-        sampleData.length(), partNumber, uploadID);
-    ozoneOutputStream.write(string2Bytes(sampleData), 0, sampleData.length());
-    ozoneOutputStream.close();
-
-    OmMultipartCommitUploadPartInfo commitUploadPartInfo = ozoneOutputStream
-        .getCommitUploadPartInfo();
-
-    assertNotNull(commitUploadPartInfo);
-    String partName = commitUploadPartInfo.getPartName();
-    assertNotNull(commitUploadPartInfo.getPartName());
-
-    // Overwrite the part by creating part key with same part number
-    // and same content.
-    ozoneOutputStream = bucket.createMultipartKey(keyName,
-        sampleData.length(), partNumber, uploadID);
-    ozoneOutputStream.write(string2Bytes(sampleData), 0, "name".length());
-    ozoneOutputStream.close();
-
-    commitUploadPartInfo = ozoneOutputStream
-        .getCommitUploadPartInfo();
-
-    assertNotNull(commitUploadPartInfo);
-    assertNotNull(commitUploadPartInfo.getPartName());
-
-    // AWS S3 for same content generates same partName during upload part.
-    // In AWS S3 ETag is generated from md5sum. In Ozone right now we
-    // don't do this. For now to make things work for large file upload
-    // through aws s3 cp, the partName are generated in a predictable fashion.
-    // So, when a part is override partNames will still be same irrespective
-    // of content in ozone s3. This will make S3 Mpu completeMPU pass when
-    // comparing part names and large file uploads work using aws cp.
-    assertEquals("Part names should be same", partName,
-        commitUploadPartInfo.getPartName());
-  }
-
-  @Test
   public void testNoSuchUploadError() throws Exception {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
@@ -2461,7 +2526,9 @@
     bucket.addAcl(acl3);
     bucket.addAcl(acl4);
 
-    doMultipartUpload(bucket, keyName, (byte)98);
+    ReplicationConfig replication = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.ONE);
+    doMultipartUpload(bucket, keyName, (byte)98, replication);
     OzoneObj keyObj = OzoneObjInfo.Builder.newBuilder()
         .setBucketName(bucketName)
         .setVolumeName(volumeName).setKeyName(keyName)
@@ -2507,7 +2574,7 @@
     OzoneBucket bucket2 = client.getObjectStore().getVolume(volumeName)
         .getBucket(bucketName);
     try {
-      initiateMultipartUpload(bucket2, keyName2, ReplicationType.RATIS, THREE);
+      initiateMultipartUpload(bucket2, keyName2, anyReplication());
       fail("User without permission should fail");
     } catch (Exception e) {
       assertTrue(e instanceof OMException);
@@ -2530,7 +2597,7 @@
     store.addAcl(bucketObj, acl9);
     store.addAcl(bucketObj, acl10);
     String uploadId = initiateMultipartUpload(bucket2, keyName2,
-        ReplicationType.RATIS, THREE);
+        anyReplication());
 
     // Upload part
     byte[] data = generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, (byte)1);
@@ -2552,8 +2619,10 @@
     }
   }
 
-  @Test
-  public void testMultipartUploadOverride() throws Exception {
+  @ParameterizedTest
+  @MethodSource("replicationConfigs")
+  void testMultipartUploadOverride(ReplicationConfig replication)
+      throws Exception {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
     String keyName = UUID.randomUUID().toString();
@@ -2563,11 +2632,11 @@
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
 
-    doMultipartUpload(bucket, keyName, (byte)96);
+    doMultipartUpload(bucket, keyName, (byte)96, replication);
 
     // Initiate Multipart upload again, now we should read latest version, as
     // read always reads latest blocks.
-    doMultipartUpload(bucket, keyName, (byte)97);
+    doMultipartUpload(bucket, keyName, (byte)97, replication);
 
   }
 
@@ -2584,8 +2653,8 @@
     OzoneBucket bucket = volume.getBucket(bucketName);
 
     // Initiate multipart upload
-    String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
-        ONE);
+    String uploadID = initiateMultipartUpload(bucket, keyName,
+        anyReplication());
 
     // Upload Parts
     Map<Integer, String> partsMap = new TreeMap<>();
@@ -2617,8 +2686,8 @@
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
 
-    String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
-        ONE);
+    String uploadID = initiateMultipartUpload(bucket, keyName,
+        anyReplication());
 
     // We have not uploaded any parts, but passing some list it should throw
     // error.
@@ -2642,8 +2711,9 @@
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
 
-    String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
-        ONE);
+    ReplicationConfig replication = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.ONE);
+    String uploadID = initiateMultipartUpload(bucket, keyName, replication);
 
     uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
     // We have not uploaded any parts, but passing some list it should throw
@@ -2667,8 +2737,9 @@
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
 
-    String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
-        ONE);
+    ReplicationConfig replication = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.ONE);
+    String uploadID = initiateMultipartUpload(bucket, keyName, replication);
 
     uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
     // We have not uploaded any parts, but passing some list it should throw
@@ -2696,7 +2767,7 @@
   }
 
   @Test
-  public void testAbortUploadFailWithInProgressPartUpload() throws Exception {
+  void testAbortUploadFailWithInProgressPartUpload() throws Exception {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
     String keyName = UUID.randomUUID().toString();
@@ -2707,7 +2778,7 @@
     OzoneBucket bucket = volume.getBucket(bucketName);
 
     OmMultipartInfo omMultipartInfo = bucket.initiateMultipartUpload(keyName,
-        RATIS, ONE);
+        anyReplication());
 
     Assert.assertNotNull(omMultipartInfo.getUploadID());
 
@@ -2731,7 +2802,7 @@
   }
 
   @Test
-  public void testCommitPartAfterCompleteUpload() throws Exception {
+  void testCommitPartAfterCompleteUpload() throws Exception {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
     String keyName = UUID.randomUUID().toString();
@@ -2742,7 +2813,7 @@
     OzoneBucket bucket = volume.getBucket(bucketName);
 
     OmMultipartInfo omMultipartInfo = bucket.initiateMultipartUpload(keyName,
-        RATIS, ONE);
+        anyReplication());
 
     Assert.assertNotNull(omMultipartInfo.getUploadID());
 
@@ -2804,8 +2875,8 @@
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
 
-    String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
-        ONE);
+    String uploadID = initiateMultipartUpload(bucket, keyName,
+        anyReplication());
     bucket.abortMultipartUpload(keyName, uploadID);
   }
 
@@ -2820,14 +2891,16 @@
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
 
-    String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
-        ONE);
+    String uploadID = initiateMultipartUpload(bucket, keyName,
+        anyReplication());
     uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
     bucket.abortMultipartUpload(keyName, uploadID);
   }
 
-  @Test
-  public void testListMultipartUploadParts() throws Exception {
+  @ParameterizedTest
+  @MethodSource("replicationConfigs")
+  void testListMultipartUploadParts(ReplicationConfig replication)
+      throws Exception {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
     String keyName = UUID.randomUUID().toString();
@@ -2838,8 +2911,7 @@
     OzoneBucket bucket = volume.getBucket(bucketName);
 
     Map<Integer, String> partsMap = new TreeMap<>();
-    String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
-        ONE);
+    String uploadID = initiateMultipartUpload(bucket, keyName, replication);
     String partName1 = uploadPart(bucket, keyName, uploadID, 1,
         generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, (byte)97));
     partsMap.put(1, partName1);
@@ -2855,10 +2927,9 @@
     OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
         bucket.listParts(keyName, uploadID, 0, 3);
 
-    Assert.assertEquals(RATIS,
-        ozoneMultipartUploadPartListParts.getReplicationType());
-    Assert.assertEquals(3,
-        ozoneMultipartUploadPartListParts.getPartInfoList().size());
+    Assert.assertEquals(
+        replication,
+        ozoneMultipartUploadPartListParts.getReplicationConfig());
 
     Assert.assertEquals(partsMap.get(ozoneMultipartUploadPartListParts
             .getPartInfoList().get(0).getPartNumber()),
@@ -2876,9 +2947,11 @@
     Assert.assertFalse(ozoneMultipartUploadPartListParts.isTruncated());
   }
 
-  @Test
-  public void testListMultipartUploadPartsWithContinuation()
-      throws Exception {
+  @ParameterizedTest
+  @MethodSource("replicationConfigs")
+  void testListMultipartUploadPartsWithContinuation(
+      ReplicationConfig replication) throws Exception {
+
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
     String keyName = UUID.randomUUID().toString();
@@ -2889,8 +2962,7 @@
     OzoneBucket bucket = volume.getBucket(bucketName);
 
     Map<Integer, String> partsMap = new TreeMap<>();
-    String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
-        ONE);
+    String uploadID = initiateMultipartUpload(bucket, keyName, replication);
     String partName1 = uploadPart(bucket, keyName, uploadID, 1,
         generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, (byte)97));
     partsMap.put(1, partName1);
@@ -2906,8 +2978,8 @@
     OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
         bucket.listParts(keyName, uploadID, 0, 2);
 
-    Assert.assertEquals(RATIS,
-        ozoneMultipartUploadPartListParts.getReplicationType());
+    Assert.assertEquals(replication,
+        ozoneMultipartUploadPartListParts.getReplicationConfig());
 
     Assert.assertEquals(2,
         ozoneMultipartUploadPartListParts.getPartInfoList().size());
@@ -2994,8 +3066,8 @@
     OzoneBucket bucket = volume.getBucket(bucketName);
 
 
-    String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
-        ONE);
+    String uploadID = initiateMultipartUpload(bucket, keyName,
+        anyReplication());
     uploadPart(bucket, keyName, uploadID, 1,
         generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, (byte)97));
 
@@ -3007,8 +3079,6 @@
 
     Assert.assertEquals(0,
         ozoneMultipartUploadPartListParts.getPartInfoList().size());
-    Assert.assertEquals(RATIS,
-        ozoneMultipartUploadPartListParts.getReplicationType());
 
     // As we don't have any parts with greater than partNumberMarker and list
     // is not truncated, so it should return false here.
@@ -3365,11 +3435,11 @@
     return chars;
   }
 
-  private void doMultipartUpload(OzoneBucket bucket, String keyName, byte val)
+  private void doMultipartUpload(OzoneBucket bucket, String keyName, byte val,
+      ReplicationConfig replication)
       throws Exception {
     // Initiate Multipart upload request
-    String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType
-        .RATIS, THREE);
+    String uploadID = initiateMultipartUpload(bucket, keyName, replication);
 
     // Upload parts
     Map<Integer, String> partsMap = new TreeMap<>();
@@ -3402,9 +3472,8 @@
     OzoneInputStream inputStream = bucket.readKey(keyName);
     inputStream.read(fileContent);
 
-    Assert.assertTrue(verifyRatisReplication(bucket.getVolumeName(),
-        bucket.getName(), keyName, ReplicationType.RATIS,
-        THREE));
+    verifyReplication(bucket.getVolumeName(), bucket.getName(), keyName,
+        replication);
 
     StringBuilder sb = new StringBuilder(length);
 
@@ -3431,10 +3500,9 @@
   }
 
   private String initiateMultipartUpload(OzoneBucket bucket, String keyName,
-      ReplicationType replicationType, ReplicationFactor replicationFactor)
-      throws Exception {
+      ReplicationConfig replicationConfig) throws Exception {
     OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
-        replicationType, replicationFactor);
+        replicationConfig);
 
     String uploadID = multipartInfo.getUploadID();
     Assert.assertNotNull(uploadID);
@@ -3551,9 +3619,8 @@
     OzoneInputStream is = bucket.readKey(keyName);
     byte[] fileContent = new byte[text.getBytes(UTF_8).length];
     is.read(fileContent);
-    Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
-        keyName, RATIS,
-        ONE));
+    verifyReplication(volumeName, bucketName, keyName,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE));
     Assert.assertEquals(text, new String(fileContent, UTF_8));
 
     //Step 4
@@ -3631,9 +3698,8 @@
     OzoneInputStream is = bucket.readKey(keyName);
     byte[] fileContent = new byte[text.getBytes(UTF_8).length];
     is.read(fileContent);
-    Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
-        keyName, RATIS,
-        ONE));
+    verifyReplication(volumeName, bucketName, keyName,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE));
     Assert.assertEquals(text, new String(fileContent, UTF_8));
 
     //Step 4
@@ -3798,4 +3864,53 @@
     createRequiredForVersioningTest(volumeName, bucketName, keyName, true);
     checkExceptedResultForVersioningTest(volumeName, bucketName, keyName, 2);
   }
+
+  @Test
+  public void testSetECReplicationConfigOnBucket()
+      throws IOException {
+    String volumeName = UUID.randomUUID().toString();
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    OzoneBucket bucket = getBucket(volume);
+    ReplicationConfig currentReplicationConfig = bucket.getReplicationConfig();
+    Assert.assertEquals(
+        StandaloneReplicationConfig.getInstance(
+            HddsProtos.ReplicationFactor.ONE),
+        currentReplicationConfig);
+    ECReplicationConfig ecReplicationConfig =
+        new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS, 1024);
+    bucket.setReplicationConfig(ecReplicationConfig);
+
+    // Get the bucket and check the updated config.
+    bucket = volume.getBucket(bucket.getName());
+
+    Assert.assertEquals(ecReplicationConfig, bucket.getReplicationConfig());
+
+    RatisReplicationConfig ratisReplicationConfig =
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+    bucket.setReplicationConfig(ratisReplicationConfig);
+
+    // Get the bucket and check the updated config.
+    bucket = volume.getBucket(bucket.getName());
+
+    Assert.assertEquals(ratisReplicationConfig, bucket.getReplicationConfig());
+
+    //Reset replication config back.
+    bucket.setReplicationConfig(currentReplicationConfig);
+  }
+
+  private OzoneBucket getBucket(OzoneVolume volume) throws IOException {
+    String bucketName = UUID.randomUUID().toString();
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setVersioning(true).setDefaultReplicationConfig(
+        new DefaultReplicationConfig(
+            StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE)));
+    volume.createBucket(bucketName, builder.build());
+    return volume.getBucket(bucketName);
+  }
+
+  private static ReplicationConfig anyReplication() {
+    return RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE);
+  }
 }
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/ECStreamTestUtil.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/ECStreamTestUtil.java
new file mode 100644
index 0000000..0503dbe
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/ECStreamTestUtil.java
@@ -0,0 +1,408 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.ozone.erasurecode.rawcoder.util.CodecUtil;
+import org.apache.ratis.util.Preconditions;
+import org.junit.Assert;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SplittableRandom;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class providing methods useful in EC tests.
+ */
+public final class ECStreamTestUtil {
+
+  private ECStreamTestUtil() {
+  }
+
+  public static OmKeyLocationInfo createKeyInfo(ReplicationConfig repConf,
+      long blockLength, Map<DatanodeDetails, Integer> dnMap) {
+
+    Pipeline pipeline = Pipeline.newBuilder()
+        .setState(Pipeline.PipelineState.CLOSED)
+        .setId(PipelineID.randomId())
+        .setNodes(new ArrayList<>(dnMap.keySet()))
+        .setReplicaIndexes(dnMap)
+        .setReplicationConfig(repConf)
+        .build();
+
+    OmKeyLocationInfo keyInfo = new OmKeyLocationInfo.Builder()
+        .setBlockID(new BlockID(1, 1))
+        .setLength(blockLength)
+        .setOffset(0)
+        .setPipeline(pipeline)
+        .setPartNumber(0)
+        .build();
+    return keyInfo;
+  }
+
+  public static OmKeyLocationInfo createKeyInfo(ReplicationConfig repConf,
+      int nodeCount, long blockLength) {
+    Map<DatanodeDetails, Integer> datanodes = new HashMap<>();
+    for (int i = 0; i < nodeCount; i++) {
+      datanodes.put(MockDatanodeDetails.randomDatanodeDetails(), i + 1);
+    }
+    return createKeyInfo(repConf, blockLength, datanodes);
+  }
+
+  /**
+   * Fill / Pad the remaining space in a buffer with zeros.
+   * @param buf
+   */
+  public static void zeroFill(ByteBuffer buf) {
+    byte[] a = buf.array();
+    Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+    buf.position(buf.limit());
+  }
+
+  /**
+   * Given a List of ByteBuffers, write length of random bytes from the given
+   * Random generator to the byte buffers. The data is striped across the
+   * buffers in stripeSize chunks.
+   * When the length of data has been written, the buffer limits are set to
+   * their final positions.
+   *
+   * @param bufs The list of buffers to fill with random data
+   * @param stripeSize The stripe size to use
+   * @param rand The random generator to use
+   * @param length The length of data to write.
+   */
+  public static void randomFill(ByteBuffer[] bufs, int stripeSize,
+      SplittableRandom rand, int length) {
+    Preconditions.assertTrue(totalSpaceAvailable(bufs) >= length);
+    int remaining = length;
+    while (remaining > 0) {
+      for (ByteBuffer b : bufs) {
+        int toWrite = Math.min(stripeSize, remaining);
+        for (int i = 0; i < toWrite; i++) {
+          b.put((byte) rand.nextInt(255));
+        }
+        remaining -= toWrite;
+      }
+    }
+    // Set the buffer limits to the final position
+    for (ByteBuffer b : bufs) {
+      b.limit(b.position());
+    }
+  }
+
+  public static void randomFill(ByteBuffer buf, SplittableRandom rand) {
+    while (buf.remaining() > 0) {
+      buf.put((byte) rand.nextInt(255));
+    }
+  }
+
+  private static int totalSpaceAvailable(ByteBuffer[] bufs) {
+    int space = 0;
+    for (ByteBuffer b : bufs) {
+      space += b.remaining();
+    }
+    return space;
+  }
+
+  /**
+   * Given a buffer which has data loaded, flip the buffer and ensure it matches
+   * byte for byte the next series of bytes from the Random generator.
+   * @param b Byte Buffers containing data
+   * @param rand The random generator
+   */
+  public static void assertBufferMatches(ByteBuffer b, SplittableRandom rand) {
+    b.flip();
+    int i = 0;
+    while (b.hasRemaining()) {
+      i++;
+      Assert.assertEquals("Failed on iteration " + i,
+          (byte)rand.nextInt(255), b.get());
+    }
+  }
+
+  /**
+   * Given a List of ByteBuffers and the RepConfig, encode the parity buffers
+   * from the data buffers. The data buffers should be passed "as is" after
+   * reading data. That is, the position will be at the last data byte read in
+   * or the buffer limit.
+   * The data buffers and parity will be returned "ready to read" with the
+   * position reset to zero.
+   * @param data List of data buffers
+   * @param ecConfig The ECReplicationConfig.
+   * @return List of encoded parity buffers.
+   * @throws IOException
+   */
+  public static ByteBuffer[] generateParity(ByteBuffer[] data,
+      ECReplicationConfig ecConfig) throws IOException {
+    // First data buffer dictates the size
+    int cellSize = data[0].limit();
+    data[0].flip();
+    // Store the positions of the remaining data buffers so we can restore them
+    int[] dataLimits = new int[data.length];
+    for (int i = 1; i < data.length; i++) {
+      dataLimits[i] = data[i].limit();
+      data[i].limit(cellSize);
+      zeroFill(data[i]);
+      data[i].flip();
+    }
+    ByteBuffer[] parity = new ByteBuffer[ecConfig.getParity()];
+    for (int i = 0; i < ecConfig.getParity(); i++) {
+      parity[i] = ByteBuffer.allocate(cellSize);
+    }
+    RawErasureEncoder encoder =
+        CodecUtil.createRawEncoderWithFallback(ecConfig);
+    encoder.encode(data, parity);
+
+    data[0].flip();
+    for (int i = 1; i < data.length; i++) {
+      data[i].limit(dataLimits[i]);
+      data[i].position(0);
+    }
+    return parity;
+  }
+
+  /**
+   * Returns a new map containing a random DatanodeDetails for each index in
+   * inputs.
+   * @param idxs A list of indexes to add to the map
+   * @return A map of DatanodeDetails to index.
+   */
+  public static Map<DatanodeDetails, Integer> createIndexMap(int... idxs) {
+    Map<DatanodeDetails, Integer> map = new HashMap<>();
+    for (int i : idxs) {
+      map.put(MockDatanodeDetails.randomDatanodeDetails(), i);
+    }
+    return map;
+  }
+
+
+  /**
+   * A stream factory which can be used in tests to provide TestBlockStream
+   * instances.
+   */
+  public static class TestBlockInputStreamFactory implements
+      BlockInputStreamFactory {
+
+    private Map<Integer, TestBlockInputStream> blockStreams =
+        new LinkedHashMap<>();
+    private List<ByteBuffer> blockStreamData;
+    // List of EC indexes that should fail immediately on read
+    private List<Integer> failIndexes = new ArrayList<>();
+
+    private Pipeline currentPipeline;
+
+    public synchronized
+        List<ECStreamTestUtil.TestBlockInputStream> getBlockStreams() {
+      return blockStreams.values().stream().collect(Collectors.toList());
+    }
+
+    public synchronized Set<Integer> getStreamIndexes() {
+      return blockStreams.keySet();
+    }
+
+    public synchronized ECStreamTestUtil.TestBlockInputStream getBlockStream(
+        int ecIndex) {
+      return blockStreams.get(ecIndex);
+    }
+
+    public synchronized void setBlockStreamData(List<ByteBuffer> bufs) {
+      this.blockStreamData = bufs;
+    }
+
+    public synchronized void setCurrentPipeline(Pipeline pipeline) {
+      this.currentPipeline = pipeline;
+    }
+
+    public synchronized void setFailIndexes(List<Integer> fail) {
+      failIndexes.addAll(fail);
+    }
+
+    public synchronized BlockExtendedInputStream create(
+        ReplicationConfig repConfig,
+        OmKeyLocationInfo blockInfo, Pipeline pipeline,
+        Token<OzoneBlockTokenIdentifier> token, boolean verifyChecksum,
+        XceiverClientFactory xceiverFactory,
+        Function<BlockID, Pipeline> refreshFunction) {
+
+      int repInd = currentPipeline.getReplicaIndex(pipeline.getNodes().get(0));
+      TestBlockInputStream stream = new TestBlockInputStream(
+          blockInfo.getBlockID(), blockInfo.getLength(),
+          blockStreamData.get(repInd - 1), repInd);
+      if (failIndexes.contains(repInd)) {
+        stream.setShouldError(true);
+      }
+      blockStreams.put(repInd, stream);
+      return stream;
+    }
+  }
+
+  /**
+   * A block stream that returns data from the provided ByteBuffer. Intended to
+   * be used in tests, rather than reading from a real block stream.
+   */
+  public static class TestBlockInputStream extends BlockExtendedInputStream {
+
+    private ByteBuffer data;
+    private boolean closed = false;
+    private BlockID blockID;
+    private long length;
+    private boolean shouldError = false;
+    private int shouldErrorPosition = 0;
+    private boolean shouldErrorOnSeek = false;
+    private IOException errorToThrow = null;
+    private int ecReplicaIndex = 0;
+    private static final byte EOF = -1;
+
+    TestBlockInputStream(BlockID blockId, long blockLen, ByteBuffer data) {
+      this(blockId, blockLen, data, 0);
+    }
+
+    TestBlockInputStream(BlockID blockId, long blockLen, ByteBuffer data,
+        int replicaIndex) {
+      this.blockID = blockId;
+      this.length = blockLen;
+      this.data = data;
+      this.ecReplicaIndex = replicaIndex;
+      data.position(0);
+    }
+
+    public boolean isClosed() {
+      return closed;
+    }
+
+    public void setShouldErrorOnSeek(boolean val) {
+      this.shouldErrorOnSeek = val;
+    }
+
+    public void setShouldError(boolean val) {
+      shouldError = val;
+      shouldErrorPosition = 0;
+    }
+
+    public void setShouldError(boolean val, int position,
+        IOException errorThrowable) {
+      this.shouldError = val;
+      this.shouldErrorPosition = position;
+      this.errorToThrow = errorThrowable;
+    }
+
+    public int getEcReplicaIndex() {
+      return ecReplicaIndex;
+    }
+
+    @Override
+    public BlockID getBlockID() {
+      return blockID;
+    }
+
+    @Override
+    public long getLength() {
+      return length;
+    }
+
+    @Override
+    public long getRemaining() {
+      return getLength() - getPos();
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len)
+        throws IOException {
+      return read(ByteBuffer.wrap(b, off, len));
+    }
+
+    @Override
+    public int read(ByteBuffer buf) throws IOException {
+      if (shouldError && data.position() >= shouldErrorPosition) {
+        throwError();
+      }
+      if (getRemaining() == 0) {
+        return EOF;
+      }
+      int toRead = (int)Math.min(buf.remaining(), getRemaining());
+      for (int i = 0; i < toRead; i++) {
+        if (shouldError && data.position() >= shouldErrorPosition) {
+          throwError();
+        }
+        buf.put(data.get());
+      }
+      return toRead;
+    };
+
+    private void throwError() throws IOException {
+      if (errorToThrow != null) {
+        throw errorToThrow;
+      } else {
+        throw new IOException("Simulated error reading block");
+      }
+    }
+
+    @Override
+    protected int readWithStrategy(ByteReaderStrategy strategy) throws
+        IOException {
+      throw new IOException("Should not be called");
+    }
+
+    @Override
+    public void close() {
+      closed = true;
+    }
+
+    @Override
+    public void unbuffer() {
+    }
+
+    @Override
+    public long getPos() {
+      return data.position();
+    }
+
+    @Override
+    public void seek(long pos) throws IOException {
+      if (shouldErrorOnSeek) {
+        throw new IOException("Simulated exception");
+      }
+      data.position((int)pos);
+    }
+
+  }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStreamFactoryImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStreamFactoryImpl.java
new file mode 100644
index 0000000..8532835
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStreamFactoryImpl.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.junit.Test;
+import org.junit.Assert;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Tests for BlockInputStreamFactoryImpl.
+ */
+public class TestBlockInputStreamFactoryImpl {
+
+  @Test
+  public void testNonECGivesBlockInputStream() {
+    BlockInputStreamFactory factory = new BlockInputStreamFactoryImpl();
+    ReplicationConfig repConfig =
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+
+    OmKeyLocationInfo blockInfo = createKeyLocationInfo(repConfig, 3,
+        1024 * 1024 * 10);
+
+    BlockExtendedInputStream stream =
+        factory.create(repConfig, blockInfo, blockInfo.getPipeline(),
+            blockInfo.getToken(), true, null, null);
+    Assert.assertTrue(stream instanceof BlockInputStream);
+    Assert.assertEquals(stream.getBlockID(), blockInfo.getBlockID());
+    Assert.assertEquals(stream.getLength(), blockInfo.getLength());
+  }
+
+  @Test
+  public void testECGivesECBlockInputStream() {
+    BlockInputStreamFactory factory = new BlockInputStreamFactoryImpl();
+    ReplicationConfig repConfig =
+        new ECReplicationConfig(3, 2);
+
+    OmKeyLocationInfo blockInfo =
+        createKeyLocationInfo(repConfig, 5, 1024 * 1024 * 10);
+
+    BlockExtendedInputStream stream =
+        factory.create(repConfig, blockInfo, blockInfo.getPipeline(),
+            blockInfo.getToken(), true, null, null);
+    Assert.assertTrue(stream instanceof ECBlockInputStreamProxy);
+    Assert.assertEquals(stream.getBlockID(), blockInfo.getBlockID());
+    Assert.assertEquals(stream.getLength(), blockInfo.getLength());
+  }
+
+  private OmKeyLocationInfo createKeyLocationInfo(ReplicationConfig repConf,
+      long blockLength, Map<DatanodeDetails, Integer> dnMap) {
+
+    Pipeline pipeline = Pipeline.newBuilder()
+        .setState(Pipeline.PipelineState.CLOSED)
+        .setId(PipelineID.randomId())
+        .setNodes(new ArrayList<>(dnMap.keySet()))
+        .setReplicaIndexes(dnMap)
+        .setReplicationConfig(repConf)
+        .build();
+
+    OmKeyLocationInfo keyInfo = new OmKeyLocationInfo.Builder()
+        .setBlockID(new BlockID(1, 1))
+        .setLength(blockLength)
+        .setOffset(0)
+        .setPipeline(pipeline)
+        .setPartNumber(0)
+        .build();
+    return keyInfo;
+  }
+
+  private OmKeyLocationInfo createKeyLocationInfo(ReplicationConfig repConf,
+      int nodeCount, long blockLength) {
+    Map<DatanodeDetails, Integer> datanodes = new HashMap<>();
+    for (int i = 0; i < nodeCount; i++) {
+      datanodes.put(MockDatanodeDetails.randomDatanodeDetails(), i + 1);
+    }
+    return createKeyLocationInfo(repConf, blockLength, datanodes);
+  }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java
index c39598e..82f494f 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+
 import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
 import org.apache.hadoop.hdds.scm.storage.ChunkInputStream;
 import org.apache.hadoop.ozone.client.io.KeyInputStream;
@@ -57,7 +58,8 @@
 
     KeyInputStream keyInputStream = getKeyInputStream(keyName);
 
-    BlockInputStream block0Stream = keyInputStream.getBlockStreams().get(0);
+    BlockInputStream block0Stream =
+        (BlockInputStream)keyInputStream.getBlockStreams().get(0);
     block0Stream.initialize();
 
     ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0);
@@ -121,7 +123,8 @@
 
     try (KeyInputStream keyInputStream = getKeyInputStream(keyName)) {
 
-      BlockInputStream block0Stream = keyInputStream.getBlockStreams().get(0);
+      BlockInputStream block0Stream =
+          (BlockInputStream)keyInputStream.getBlockStreams().get(0);
       block0Stream.initialize();
 
       ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java
new file mode 100644
index 0000000..17a0a6f
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java
@@ -0,0 +1,522 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BadDataLocationException;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+/**
+ * Tests for ECBlockInputStream.
+ */
+public class TestECBlockInputStream {
+
+  private static final int ONEMB = 1024 * 1024;
+
+  private ECReplicationConfig repConfig;
+  private TestBlockInputStreamFactory streamFactory;
+
+  @Before
+  public void setup() {
+    repConfig = new ECReplicationConfig(3, 2,
+        ECReplicationConfig.EcCodec.RS, ONEMB);
+    streamFactory = new TestBlockInputStreamFactory();
+  }
+
+  @Test
+  public void testSufficientLocations() {
+    // EC-3-2, 5MB block, so all 3 data locations are needed
+    OmKeyLocationInfo keyInfo = ECStreamTestUtil
+        .createKeyInfo(repConfig, 5, 5 * ONEMB);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // EC-3-2, very large block, so all 3 data locations are needed
+    keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, 5, 5000 * ONEMB);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+    // EC-3-2, 1 byte short of 1MB with 1 location
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB - 1, dnMap);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // EC-3-2, 5MB blocks, only 2 locations passed so we do not have sufficient
+    // locations.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+
+    // EC-3-2, 5MB blocks, only 1 data and 2 parity locations present. For now
+    // this will fail as we don't support reconstruction reads yet.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 4);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 5);
+    keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStreamLessThanCell()
+      throws IOException {
+    ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB - 100);
+
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.read(buf);
+      // We expect only 1 block stream and it should have a length passed of
+      // ONEMB - 100.
+      List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB - 100, streams.get(0).getLength());
+    }
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStreamTwoCells()
+      throws IOException {
+    ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB + 100);
+
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.read(buf);
+      List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(100, streams.get(1).getLength());
+    }
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStreamThreeCells()
+      throws IOException {
+    ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 2 * ONEMB + 100);
+
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.read(buf);
+      List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(ONEMB, streams.get(1).getLength());
+      Assert.assertEquals(100, streams.get(2).getLength());
+    }
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStreamThreeFullAndPartialStripe()
+      throws IOException {
+    ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 10 * ONEMB + 100);
+
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.read(buf);
+      List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(4 * ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(3 * ONEMB + 100, streams.get(1).getLength());
+      Assert.assertEquals(3 * ONEMB, streams.get(2).getLength());
+    }
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStreamSingleFullCell()
+      throws IOException {
+    ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB);
+
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.read(buf);
+      List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB, streams.get(0).getLength());
+    }
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStreamSeveralFullCells()
+      throws IOException {
+    ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 9 * ONEMB);
+
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.read(buf);
+      List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(3 * ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(3 * ONEMB, streams.get(1).getLength());
+      Assert.assertEquals(3 * ONEMB, streams.get(2).getLength());
+    }
+  }
+
+  @Test
+  public void testSimpleRead() throws IOException {
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+
+      ByteBuffer buf = ByteBuffer.allocate(100);
+
+      int read = ecb.read(buf);
+      Assert.assertEquals(100, read);
+      validateBufferContents(buf, 0, 100, (byte) 0);
+      Assert.assertEquals(100, ecb.getPos());
+    }
+    for (TestBlockInputStream s : streamFactory.getBlockStreams()) {
+      Assert.assertTrue(s.isClosed());
+    }
+  }
+
+  /**
+   * This test is to ensure we can read a small key of 1 chunk or less when only
+   * the first replica index is available.
+   */
+  @Test
+  public void testSimpleReadUnderOneChunk() throws IOException {
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 1, ONEMB);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+
+      ByteBuffer buf = ByteBuffer.allocate(100);
+
+      int read = ecb.read(buf);
+      Assert.assertEquals(100, read);
+      validateBufferContents(buf, 0, 100, (byte) 0);
+      Assert.assertEquals(100, ecb.getPos());
+    }
+    for (TestBlockInputStream s : streamFactory.getBlockStreams()) {
+      Assert.assertTrue(s.isClosed());
+    }
+  }
+
+  @Test
+  public void testReadPastEOF() throws IOException {
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 50);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+
+      ByteBuffer buf = ByteBuffer.allocate(100);
+
+      int read = ecb.read(buf);
+      Assert.assertEquals(50, read);
+      read = ecb.read(buf);
+      Assert.assertEquals(read, -1);
+    }
+  }
+
+  @Test
+  public void testReadCrossingMultipleECChunkBounds() throws IOException {
+    // EC-3-2, 5MB block, so all 3 data locations are needed
+    repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+        100);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+
+      // EC Chunk size is 100 and 3-2. Create a byte buffer to read 3.5 chunks,
+      // so 350
+      ByteBuffer buf = ByteBuffer.allocate(350);
+      int read = ecb.read(buf);
+      Assert.assertEquals(350, read);
+
+      validateBufferContents(buf, 0, 100, (byte) 0);
+      validateBufferContents(buf, 100, 200, (byte) 1);
+      validateBufferContents(buf, 200, 300, (byte) 2);
+      validateBufferContents(buf, 300, 350, (byte) 0);
+
+      buf.clear();
+      read = ecb.read(buf);
+      Assert.assertEquals(350, read);
+
+      validateBufferContents(buf, 0, 50, (byte) 0);
+      validateBufferContents(buf, 50, 150, (byte) 1);
+      validateBufferContents(buf, 150, 250, (byte) 2);
+      validateBufferContents(buf, 250, 350, (byte) 0);
+
+    }
+    for (TestBlockInputStream s : streamFactory.getBlockStreams()) {
+      Assert.assertTrue(s.isClosed());
+    }
+  }
+
+  @Test(expected = EOFException.class)
+  public void testSeekPastBlockLength() throws IOException {
+    repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+        ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 100);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.seek(1000);
+    }
+  }
+
+  @Test(expected = EOFException.class)
+  public void testSeekToLength() throws IOException {
+    repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+        ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 100);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.seek(100);
+    }
+  }
+
+  @Test
+  public void testSeekToLengthZeroLengthBlock() throws IOException {
+    repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+        ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 0);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.seek(0);
+      Assert.assertEquals(0, ecb.getPos());
+      Assert.assertEquals(0, ecb.getRemaining());
+    }
+  }
+
+  @Test
+  public void testSeekToValidPosition() throws IOException {
+    repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+        ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      ecb.seek(ONEMB - 1);
+      Assert.assertEquals(ONEMB - 1, ecb.getPos());
+      Assert.assertEquals(ONEMB * 4 + 1, ecb.getRemaining());
+      // First read should read the last byte of the first chunk
+      Assert.assertEquals(0, ecb.read());
+      Assert.assertEquals(ONEMB,
+          streamFactory.getBlockStreams().get(0).position);
+      // Second read should be the first byte of the second chunk.
+      Assert.assertEquals(1, ecb.read());
+
+      // Seek to the end of the file minus one byte
+      ecb.seek(ONEMB * 5 - 1);
+      Assert.assertEquals(1, ecb.read());
+      Assert.assertEquals(ONEMB * 2,
+          streamFactory.getBlockStreams().get(1).position);
+      // Second read should be EOF as there is no data left
+      Assert.assertEquals(-1, ecb.read());
+      Assert.assertEquals(0, ecb.getRemaining());
+    }
+  }
+
+  @Test
+  public void testErrorReadingBlockReportsBadLocation() throws IOException {
+    repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+        ONEMB);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+        keyInfo, true, null, null, streamFactory)) {
+      // Read a full stripe to ensure all streams are created in the stream
+      // factory
+      ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+      int read = ecb.read(buf);
+      Assert.assertEquals(3 * ONEMB, read);
+      // Now make replication index 2 error on the next read
+      streamFactory.getBlockStreams().get(1).setThrowException(true);
+      buf.clear();
+      try {
+        ecb.read(buf);
+        Assert.fail("Exception should be thrown");
+      } catch (IOException e) {
+        Assert.assertTrue(e instanceof BadDataLocationException);
+        Assert.assertEquals(2,
+            keyInfo.getPipeline().getReplicaIndex(
+                ((BadDataLocationException) e).getFailedLocation()));
+      }
+    }
+  }
+
+  private void validateBufferContents(ByteBuffer buf, int from, int to,
+      byte val) {
+    for (int i = from; i < to; i++) {
+      Assert.assertEquals(val, buf.get(i));
+    }
+  }
+
+  private static class TestBlockInputStreamFactory implements
+      BlockInputStreamFactory {
+
+    private List<TestBlockInputStream> blockStreams = new ArrayList<>();
+
+    public synchronized List<TestBlockInputStream> getBlockStreams() {
+      return blockStreams;
+    }
+
+    public synchronized BlockExtendedInputStream create(
+        ReplicationConfig repConfig, OmKeyLocationInfo blockInfo,
+        Pipeline pipeline, Token<OzoneBlockTokenIdentifier> token,
+        boolean verifyChecksum, XceiverClientFactory xceiverFactory,
+        Function<BlockID, Pipeline> refreshFunction) {
+      TestBlockInputStream stream = new TestBlockInputStream(
+          blockInfo.getBlockID(), blockInfo.getLength(),
+          (byte)blockStreams.size());
+      blockStreams.add(stream);
+      return stream;
+    }
+  }
+
+  private static class TestBlockInputStream extends BlockExtendedInputStream {
+
+    private long position = 0;
+    private boolean closed = false;
+    private byte dataVal = 1;
+    private BlockID blockID;
+    private long length;
+    private boolean throwException = false;
+    private static final byte EOF = -1;
+
+    @SuppressWarnings("checkstyle:parameternumber")
+    TestBlockInputStream(BlockID blockId, long blockLen, byte dataVal) {
+      this.dataVal = dataVal;
+      this.blockID = blockId;
+      this.length = blockLen;
+    }
+
+    public boolean isClosed() {
+      return closed;
+    }
+
+    public void setThrowException(boolean shouldThrow) {
+      this.throwException = shouldThrow;
+    }
+
+    @Override
+    public BlockID getBlockID() {
+      return blockID;
+    }
+
+    @Override
+    public long getLength() {
+      return length;
+    }
+
+    @Override
+    public long getRemaining() {
+      return getLength() - position;
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len)
+        throws IOException {
+      return read(ByteBuffer.wrap(b, off, len));
+    }
+
+    @Override
+    public int read(ByteBuffer buf) throws IOException {
+      if (getRemaining() == 0) {
+        return EOF;
+      }
+
+      if (throwException) {
+        throw new IOException("Simulated exception");
+      }
+
+      int toRead = Math.min(buf.remaining(), (int)getRemaining());
+      for (int i = 0; i < toRead; i++) {
+        buf.put(dataVal);
+      }
+      position += toRead;
+      return toRead;
+    };
+
+    @Override
+    protected int readWithStrategy(ByteReaderStrategy strategy) throws
+        IOException {
+      throw new IOException("Should not be called");
+    }
+
+    @Override
+    public void close() {
+      closed = true;
+    }
+
+    @Override
+    public void unbuffer() {
+    }
+
+    @Override
+    public void seek(long pos) {
+      this.position = pos;
+    }
+
+    @Override
+    public long getPos() {
+      return position;
+    }
+  }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStreamProxy.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStreamProxy.java
new file mode 100644
index 0000000..fd4e8ad
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStreamProxy.java
@@ -0,0 +1,399 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.ozone.client.io.BadDataLocationException;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SplittableRandom;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Unit tests for the  ECBlockInputStreamProxy class.
+ */
+public class TestECBlockInputStreamProxy {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestECBlockInputStreamProxy.class);
+
+  private static final int ONEMB = 1024 * 1024;
+  private ECReplicationConfig repConfig;
+  private TestECBlockInputStreamFactory streamFactory;
+
+  private long randomSeed;
+  private ThreadLocalRandom random = ThreadLocalRandom.current();
+  private SplittableRandom dataGenerator;
+
+  @Before
+  public void setup() {
+    repConfig = new ECReplicationConfig(3, 2);
+    streamFactory = new TestECBlockInputStreamFactory();
+    randomSeed = random.nextLong();
+    dataGenerator = new SplittableRandom(randomSeed);
+  }
+
+  @Test
+  public void testExpectedDataLocations() {
+    Assert.assertEquals(1,
+        ECBlockInputStreamProxy.expectedDataLocations(repConfig, 1));
+    Assert.assertEquals(2,
+        ECBlockInputStreamProxy.expectedDataLocations(repConfig, ONEMB + 1));
+    Assert.assertEquals(3,
+        ECBlockInputStreamProxy.expectedDataLocations(repConfig, 3 * ONEMB));
+    Assert.assertEquals(3,
+        ECBlockInputStreamProxy.expectedDataLocations(repConfig, 10 * ONEMB));
+
+    repConfig = new ECReplicationConfig(6, 3);
+    Assert.assertEquals(1,
+        ECBlockInputStreamProxy.expectedDataLocations(repConfig, 1));
+    Assert.assertEquals(2,
+        ECBlockInputStreamProxy.expectedDataLocations(repConfig, ONEMB + 1));
+    Assert.assertEquals(3,
+        ECBlockInputStreamProxy.expectedDataLocations(repConfig, 3 * ONEMB));
+    Assert.assertEquals(6,
+        ECBlockInputStreamProxy.expectedDataLocations(repConfig, 10 * ONEMB));
+  }
+
+  @Test
+  public void testAvailableDataLocations() {
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo blockInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, 1024, dnMap);
+    Assert.assertEquals(1, ECBlockInputStreamProxy.availableDataLocations(
+        blockInfo.getPipeline(), 1));
+    Assert.assertEquals(2, ECBlockInputStreamProxy.availableDataLocations(
+        blockInfo.getPipeline(), 2));
+    Assert.assertEquals(3, ECBlockInputStreamProxy.availableDataLocations(
+        blockInfo.getPipeline(), 3));
+
+    dnMap = ECStreamTestUtil.createIndexMap(1, 4, 5);
+    blockInfo = ECStreamTestUtil.createKeyInfo(repConfig, 1024, dnMap);
+    Assert.assertEquals(1, ECBlockInputStreamProxy.availableDataLocations(
+        blockInfo.getPipeline(), 3));
+
+    dnMap = ECStreamTestUtil.createIndexMap(2, 3, 4, 5);
+    blockInfo = ECStreamTestUtil.createKeyInfo(repConfig, 1024, dnMap);
+    Assert.assertEquals(0, ECBlockInputStreamProxy.availableDataLocations(
+        blockInfo.getPipeline(), 1));
+  }
+
+  @Test
+  public void testBlockIDCanBeRetrieved() throws IOException {
+    int blockLength = 1234;
+    generateData(blockLength);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo blockInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+    try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+      Assert.assertEquals(blockInfo.getBlockID(), bis.getBlockID());
+    }
+  }
+
+  @Test
+  public void testBlockLengthCanBeRetrieved() throws IOException {
+    int blockLength = 1234;
+    generateData(blockLength);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo blockInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+    try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+      Assert.assertEquals(1234, bis.getLength());
+    }
+  }
+
+  @Test
+  public void testBlockRemainingCanBeRetrieved() throws IOException {
+    int blockLength = 12345;
+    generateData(blockLength);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo blockInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+    dataGenerator = new SplittableRandom(randomSeed);
+    ByteBuffer readBuffer = ByteBuffer.allocate(100);
+    try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+      Assert.assertEquals(12345, bis.getRemaining());
+      Assert.assertEquals(0, bis.getPos());
+      bis.read(readBuffer);
+      Assert.assertEquals(12345 - 100, bis.getRemaining());
+      Assert.assertEquals(100, bis.getPos());
+    }
+  }
+
+  @Test
+  public void testCorrectStreamCreatedDependingOnDataLocations()
+      throws IOException {
+    int blockLength = 5 * ONEMB;
+    ByteBuffer data = generateData(blockLength);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo blockInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+    try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+      // Not all locations present, so we expect on;y the "missing=true" stream
+      // to be present.
+      Assert.assertTrue(streamFactory.getStreams().containsKey(false));
+      Assert.assertFalse(streamFactory.getStreams().containsKey(true));
+    }
+
+    streamFactory = new TestECBlockInputStreamFactory();
+    streamFactory.setData(data);
+    dnMap = ECStreamTestUtil.createIndexMap(2, 3, 4, 5);
+    blockInfo = ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+    try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+      // Not all locations present, so we expect on;y the "missing=true" stream
+      // to be present.
+      Assert.assertFalse(streamFactory.getStreams().containsKey(false));
+      Assert.assertTrue(streamFactory.getStreams().containsKey(true));
+    }
+  }
+
+  @Test
+  public void testCanReadNonReconstructionToEOF()
+      throws IOException {
+    int blockLength = 5 * ONEMB;
+    generateData(blockLength);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo blockInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+    ByteBuffer readBuffer = ByteBuffer.allocate(100);
+    dataGenerator = new SplittableRandom(randomSeed);
+    try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+      while (true) {
+        int read = bis.read(readBuffer);
+        ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+        readBuffer.clear();
+        if (read < 100) {
+          break;
+        }
+      }
+      readBuffer.clear();
+      int read = bis.read(readBuffer);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testCanReadReconstructionToEOF()
+      throws IOException {
+    int blockLength = 5 * ONEMB;
+    generateData(blockLength);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(2, 3, 4, 5);
+    OmKeyLocationInfo blockInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+    ByteBuffer readBuffer = ByteBuffer.allocate(100);
+    dataGenerator = new SplittableRandom(randomSeed);
+    try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+      while (true) {
+        int read = bis.read(readBuffer);
+        ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+        readBuffer.clear();
+        if (read < 100) {
+          break;
+        }
+      }
+      readBuffer.clear();
+      int read = bis.read(readBuffer);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testCanHandleErrorAndFailOverToReconstruction()
+      throws IOException {
+    int blockLength = 5 * ONEMB;
+    generateData(blockLength);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo blockInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+    ByteBuffer readBuffer = ByteBuffer.allocate(100);
+    DatanodeDetails badDN = blockInfo.getPipeline().getFirstNode();
+
+    dataGenerator = new SplittableRandom(randomSeed);
+    try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+      // Perform one read to get the stream created
+      int read = bis.read(readBuffer);
+      Assert.assertEquals(100, read);
+      ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+      // Setup an error to be thrown part through a read, so the dataBuffer
+      // will have been advanced by 50 bytes before the error. This tests it
+      // correctly rewinds and the same data is loaded again from the other
+      // stream.
+      streamFactory.getStreams().get(false).setShouldError(true, 151,
+          new BadDataLocationException(badDN, "Simulated Error"));
+      while (true) {
+        readBuffer.clear();
+        read = bis.read(readBuffer);
+        ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+        if (read < 100) {
+          break;
+        }
+      }
+      readBuffer.clear();
+      read = bis.read(readBuffer);
+      Assert.assertEquals(-1, read);
+      // Ensure the bad location was passed into the factory to create the
+      // reconstruction reader
+      Assert.assertEquals(badDN, streamFactory.getFailedLocations().get(0));
+    }
+  }
+
+  @Test
+  public void testCanSeekToNewPosition() throws IOException {
+    int blockLength = 5 * ONEMB;
+    generateData(blockLength);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo blockInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+    ByteBuffer readBuffer = ByteBuffer.allocate(100);
+    dataGenerator = new SplittableRandom(randomSeed);
+    try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+      // Perform one read to get the stream created
+      int read = bis.read(readBuffer);
+      Assert.assertEquals(100, read);
+
+      bis.seek(1024);
+      readBuffer.clear();
+      resetAndAdvanceDataGenerator(1024);
+      bis.read(readBuffer);
+      ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+      Assert.assertEquals(1124, bis.getPos());
+
+      // Set the non-reconstruction reader to thrown an exception on seek
+      streamFactory.getStreams().get(false).setShouldErrorOnSeek(true);
+      bis.seek(2048);
+      readBuffer.clear();
+      resetAndAdvanceDataGenerator(2048);
+      bis.read(readBuffer);
+      ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+
+      // Finally, set the recon reader to fail on seek.
+      streamFactory.getStreams().get(true).setShouldErrorOnSeek(true);
+      try {
+        bis.seek(1024);
+        Assert.fail("Seek should have raised an exception");
+      } catch (IOException e) {
+        // expected
+      }
+    }
+  }
+
+  private ByteBuffer generateData(int length) {
+    ByteBuffer data = ByteBuffer.allocate(length);
+    ECStreamTestUtil.randomFill(data, dataGenerator);
+    streamFactory.setData(data);
+    return data;
+  }
+
+  private void resetAndAdvanceDataGenerator(long position) {
+    dataGenerator = new SplittableRandom(randomSeed);
+    for (long i = 0; i < position; i++) {
+      dataGenerator.nextInt(255);
+    }
+  }
+
+  private ECBlockInputStreamProxy createBISProxy(ECReplicationConfig rConfig,
+      OmKeyLocationInfo blockInfo) {
+    return new ECBlockInputStreamProxy(
+        rConfig, blockInfo, true, null, null, streamFactory);
+  }
+
+  private static class TestECBlockInputStreamFactory
+      implements ECBlockInputStreamFactory {
+
+    private ByteBuffer data;
+
+    private Map<Boolean, ECStreamTestUtil.TestBlockInputStream> streams
+        = new HashMap<>();
+
+    private List<DatanodeDetails> failedLocations;
+
+    public void setData(ByteBuffer data) {
+      this.data = data;
+    }
+
+    public Map<Boolean, ECStreamTestUtil.TestBlockInputStream> getStreams() {
+      return streams;
+    }
+
+    public List<DatanodeDetails> getFailedLocations() {
+      return failedLocations;
+    }
+
+    @Override
+    public BlockExtendedInputStream create(boolean missingLocations,
+        List<DatanodeDetails> failedDatanodes,
+        ReplicationConfig repConfig, OmKeyLocationInfo blockInfo,
+        boolean verifyChecksum, XceiverClientFactory xceiverFactory,
+        Function<BlockID, Pipeline> refreshFunction) {
+      this.failedLocations = failedDatanodes;
+      ByteBuffer wrappedBuffer =
+          ByteBuffer.wrap(data.array(), 0, data.capacity());
+      ECStreamTestUtil.TestBlockInputStream is =
+          new ECStreamTestUtil.TestBlockInputStream(blockInfo.getBlockID(),
+              blockInfo.getLength(), wrappedBuffer);
+      streams.put(missingLocations, is);
+      return is;
+    }
+  }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedInputStream.java
new file mode 100644
index 0000000..ee3f50a
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedInputStream.java
@@ -0,0 +1,379 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.SplittableRandom;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.ozone.client.rpc.read.ECStreamTestUtil.generateParity;
+
+/**
+ * Test for the ECBlockReconstructedInputStream class.
+ */
+public class TestECBlockReconstructedInputStream {
+
+  private ECReplicationConfig repConfig;
+  private ECStreamTestUtil.TestBlockInputStreamFactory streamFactory;
+  private long randomSeed;
+  private ThreadLocalRandom random = ThreadLocalRandom.current();
+  private SplittableRandom dataGenerator;
+  private ByteBufferPool bufferPool = new ElasticByteBufferPool();
+  private ExecutorService ecReconstructExecutor =
+      Executors.newFixedThreadPool(3);
+
+  @Before
+  public void setup() throws IOException {
+    repConfig = new ECReplicationConfig(3, 2);
+    streamFactory = new ECStreamTestUtil.TestBlockInputStreamFactory();
+
+    randomSeed = random.nextLong();
+    dataGenerator = new SplittableRandom(randomSeed);
+  }
+
+  @After
+  public void teardown() {
+    ecReconstructExecutor.shutdownNow();
+  }
+
+  private ECBlockReconstructedStripeInputStream createStripeInputStream(
+      Map<DatanodeDetails, Integer> dnMap, long blockLength) {
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+        null, null, streamFactory, bufferPool, ecReconstructExecutor);
+  }
+
+  @Test
+  public void testBlockLengthReturned() throws IOException {
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(4, 5);
+    try (ECBlockReconstructedStripeInputStream stripeStream
+        = createStripeInputStream(dnMap, 12345L)) {
+      try (ECBlockReconstructedInputStream stream =
+          new ECBlockReconstructedInputStream(repConfig, bufferPool,
+              stripeStream)) {
+        Assert.assertEquals(12345L, stream.getLength());
+      }
+    }
+  }
+
+  @Test
+  public void testBlockIDReturned() throws IOException {
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 4, 5);
+    try (ECBlockReconstructedStripeInputStream stripeStream
+        = createStripeInputStream(dnMap, 12345L)) {
+      try (ECBlockReconstructedInputStream stream =
+          new ECBlockReconstructedInputStream(repConfig, bufferPool,
+              stripeStream)) {
+        Assert.assertEquals(new BlockID(1, 1), stream.getBlockID());
+      }
+    }
+  }
+
+  @Test
+  public void testReadDataByteBufferMultipleStripes() throws IOException {
+    int readBufferSize = random.nextInt(4096);
+    // 3 stripes and a partial chunk
+    int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+        + repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+        repConfig.getEcChunkSize() * 4);
+    ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+        dataGenerator, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+    try (ECBlockReconstructedStripeInputStream stripeStream
+        = createStripeInputStream(dnMap, blockLength)) {
+      try (ECBlockReconstructedInputStream stream =
+          new ECBlockReconstructedInputStream(repConfig, bufferPool,
+              stripeStream)) {
+        ByteBuffer b = ByteBuffer.allocate(readBufferSize);
+        int totalRead = 0;
+        dataGenerator = new SplittableRandom(randomSeed);
+        while (totalRead < blockLength) {
+          int expectedRead = Math.min(blockLength - totalRead, readBufferSize);
+          long read = stream.read(b);
+          totalRead += read;
+          Assert.assertEquals(expectedRead, read);
+          ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+          b.clear();
+        }
+        // Next read should be EOF
+        b.clear();
+        long read = stream.read(b);
+        Assert.assertEquals(-1, read);
+        // Seek back to zero and read again to ensure the buffers are
+        // re-allocated after being freed at the end of block.
+        stream.seek(0);
+        read = stream.read(b);
+        Assert.assertEquals(readBufferSize, read);
+        dataGenerator = new SplittableRandom(randomSeed);
+        ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+      }
+    }
+  }
+
+
+  @Test
+  public void testReadDataWithUnbuffer() throws IOException {
+    // Read buffer is 16kb + 5 bytes so it does not align with stripes exactly
+    int readBufferSize = random.nextInt(1024 * 16 + 5);
+    // 3 stripes and a partial chunk
+    int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+        + repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+        repConfig.getEcChunkSize() * 4);
+    ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+        dataGenerator, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+    try (ECBlockReconstructedStripeInputStream stripeStream
+             = createStripeInputStream(dnMap, blockLength)) {
+      try (ECBlockReconstructedInputStream stream =
+               new ECBlockReconstructedInputStream(repConfig, bufferPool,
+                   stripeStream)) {
+        ByteBuffer b = ByteBuffer.allocate(readBufferSize);
+        int totalRead = 0;
+        dataGenerator = new SplittableRandom(randomSeed);
+        while (totalRead < blockLength) {
+          int expectedRead = Math.min(blockLength - totalRead, readBufferSize);
+          long read = stream.read(b);
+          totalRead += read;
+          Assert.assertEquals(expectedRead, read);
+          ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+          b.clear();
+          stream.unbuffer();
+        }
+        // Next read should be EOF
+        b.clear();
+        long read = stream.read(b);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  @Test
+  public void testReadDataByteBufferUnderBufferSize() throws IOException {
+    int readBufferSize = 4096;
+    // Small block with less data that the read size
+    int blockLength = 1024;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 1024);
+    ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+        dataGenerator, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+    try (ECBlockReconstructedStripeInputStream stripeStream
+        = createStripeInputStream(dnMap, blockLength)) {
+      try (ECBlockReconstructedInputStream stream =
+          new ECBlockReconstructedInputStream(repConfig, bufferPool,
+              stripeStream)) {
+        ByteBuffer b = ByteBuffer.allocate(readBufferSize);
+        dataGenerator = new SplittableRandom(randomSeed);
+        long read = stream.read(b);
+        Assert.assertEquals(blockLength, read);
+        ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+        b.clear();
+        // Next read should be EOF
+        read = stream.read(b);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  @Test
+  public void testReadByteAtATime() throws IOException {
+    // 3 stripes and a partial chunk
+    int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+        + repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+        repConfig.getEcChunkSize() * 4);
+    ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+        dataGenerator, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+    try (ECBlockReconstructedStripeInputStream stripeStream
+        = createStripeInputStream(dnMap, blockLength)) {
+      try (ECBlockReconstructedInputStream stream =
+          new ECBlockReconstructedInputStream(repConfig, bufferPool,
+              stripeStream)) {
+
+        dataGenerator = new SplittableRandom(randomSeed);
+        int totalRead = 0;
+        while (true) {
+          int val = stream.read();
+          if (val == -1) {
+            break;
+          }
+          Assert.assertEquals(dataGenerator.nextInt(255), val);
+          totalRead += 1;
+        }
+        Assert.assertEquals(blockLength, totalRead);
+      }
+    }
+  }
+
+  @Test
+  public void testReadByteBuffer() throws IOException {
+    // 3 stripes and a partial chunk
+    int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+        + repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+        repConfig.getEcChunkSize() * 4);
+    ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+        dataGenerator, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+    byte[] buf = new byte[1024];
+    try (ECBlockReconstructedStripeInputStream stripeStream
+        = createStripeInputStream(dnMap, blockLength)) {
+      try (ECBlockReconstructedInputStream stream =
+          new ECBlockReconstructedInputStream(repConfig, bufferPool,
+              stripeStream)) {
+        int totalRead = 0;
+        dataGenerator = new SplittableRandom(randomSeed);
+        while (totalRead < blockLength) {
+          int expectedRead = Math.min(blockLength - totalRead, 1024);
+          long read = stream.read(buf, 0, buf.length);
+          totalRead += read;
+          Assert.assertEquals(expectedRead, read);
+          ECStreamTestUtil.assertBufferMatches(
+              ByteBuffer.wrap(buf, 0, (int)read), dataGenerator);
+        }
+        // Next read should be EOF
+        long read = stream.read(buf, 0, buf.length);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  @Test
+  public void testSeek() throws IOException {
+    int readBufferSize = repConfig.getEcChunkSize() + 1024;
+    // 3 stripes and a partial chunk
+    int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+        + repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+        repConfig.getEcChunkSize() * 4);
+    ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+        dataGenerator, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    Map<DatanodeDetails, Integer> dnMap
+        = ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+    try (ECBlockReconstructedStripeInputStream stripeStream
+        = createStripeInputStream(dnMap, blockLength)) {
+      try (ECBlockReconstructedInputStream stream =
+          new ECBlockReconstructedInputStream(repConfig, bufferPool,
+              stripeStream)) {
+        ByteBuffer b = ByteBuffer.allocate(readBufferSize);
+
+        int seekPosition = 0;
+        for (int i = 0; i < 100; i++) {
+          resetAndAdvanceDataGenerator(seekPosition);
+          long expectedRead = Math.min(stream.getRemaining(), readBufferSize);
+          long read = stream.read(b);
+          Assert.assertEquals(expectedRead, read);
+          ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+          seekPosition = random.nextInt(blockLength);
+          stream.seek(seekPosition);
+          b.clear();
+        }
+        // Seeking beyond EOF should give an error
+        try {
+          stream.seek(blockLength + 1);
+          Assert.fail("Seek beyond EOF should error");
+        } catch (IOException e) {
+          // expected
+        }
+      }
+    }
+  }
+
+  private void resetAndAdvanceDataGenerator(long position) {
+    dataGenerator = new SplittableRandom(randomSeed);
+    for (long i = 0; i < position; i++) {
+      dataGenerator.nextInt(255);
+    }
+  }
+
+
+
+  /**
+   * Return a list of num ByteBuffers of the given size.
+   * @param num Number of buffers to create
+   * @param size The size of each buffer
+   * @return
+   */
+  private ByteBuffer[] allocateBuffers(int num, int size) {
+    ByteBuffer[] bufs = new ByteBuffer[num];
+    for (int i = 0; i < num; i++) {
+      bufs[i] = ByteBuffer.allocate(size);
+    }
+    return bufs;
+  }
+
+  private void addDataStreamsToFactory(ByteBuffer[] data, ByteBuffer[] parity) {
+    List<ByteBuffer> dataStreams = new ArrayList<>();
+    for (ByteBuffer b : data) {
+      dataStreams.add(b);
+    }
+    for (ByteBuffer b : parity) {
+      dataStreams.add(b);
+    }
+    streamFactory.setBlockStreamData(dataStreams);
+  }
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
new file mode 100644
index 0000000..0eff8f9
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
@@ -0,0 +1,724 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.client.io.InsufficientLocationsException;
+import org.apache.hadoop.ozone.client.rpc.read.ECStreamTestUtil.TestBlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.rpc.read.ECStreamTestUtil.TestBlockInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.SplittableRandom;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.ozone.client.rpc.read.ECStreamTestUtil.generateParity;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */
+public class TestECBlockReconstructedStripeInputStream {
+
+  private static final int ONEMB = 1024 * 1024;
+
+  private ECReplicationConfig repConfig;
+  private ECStreamTestUtil.TestBlockInputStreamFactory streamFactory;
+  private long randomSeed;
+  private ThreadLocalRandom random = ThreadLocalRandom.current();
+  private SplittableRandom dataGen;
+  private ByteBufferPool bufferPool = new ElasticByteBufferPool();
+  private ExecutorService ecReconstructExecutor =
+      Executors.newFixedThreadPool(3);
+
+  @Before
+  public void setup() {
+    repConfig = new ECReplicationConfig(3, 2,
+        ECReplicationConfig.EcCodec.RS, ONEMB);
+    streamFactory = new ECStreamTestUtil.TestBlockInputStreamFactory();
+
+    randomSeed = random.nextLong();
+    dataGen = new SplittableRandom(randomSeed);
+  }
+
+  @After
+  public void teardown() {
+    ecReconstructExecutor.shutdownNow();
+  }
+
+  @Test
+  public void testSufficientLocations() throws IOException {
+    // One chunk, only 1 location.
+    OmKeyLocationInfo keyInfo = ECStreamTestUtil
+        .createKeyInfo(repConfig, 1, ONEMB);
+    try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+    // Two Chunks, but missing data block 2.
+    Map<DatanodeDetails, Integer> dnMap
+        = ECStreamTestUtil.createIndexMap(1, 4, 5);
+    keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB * 2, dnMap);
+    try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3.
+    dnMap = ECStreamTestUtil.createIndexMap(1, 4, 5);
+    keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb =  createInputStream(keyInfo)) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+      // Set a failed location
+      List<DatanodeDetails> failed = new ArrayList<>();
+      failed.add(keyInfo.getPipeline().getFirstNode());
+      ((ECBlockReconstructedStripeInputStream)ecb).addFailedDatanodes(failed);
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3 and parity 1.
+    dnMap = ECStreamTestUtil.createIndexMap(1, 4);
+    keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, all available but fail 3
+    dnMap = ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+      // Set a failed location
+      List<DatanodeDetails> failed = new ArrayList<>();
+      for (DatanodeDetails dn : dnMap.keySet()) {
+        failed.add(dn);
+        if (failed.size() == 3) {
+          break;
+        }
+      }
+      ((ECBlockReconstructedStripeInputStream)ecb).addFailedDatanodes(failed);
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+
+    // One chunk, indexes 2 and 3 are padding, but still reported in the
+    // container list. The other locations are missing so we should have
+    // insufficient locations.
+    dnMap = ECStreamTestUtil.createIndexMap(2, 3);
+    keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB, dnMap);
+    try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+  }
+
+  @Test
+  public void testReadFullStripesWithPartial() throws IOException {
+    // Generate the input data for 3 full stripes and generate the parity.
+    int chunkSize = repConfig.getEcChunkSize();
+    int partialStripeSize = chunkSize * 2 - 1;
+    int blockLength = chunkSize * repConfig.getData() * 3 + partialStripeSize;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+    ECStreamTestUtil.randomFill(dataBufs, chunkSize, dataGen, blockLength);
+
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(ECStreamTestUtil.createIndexMap(1, 4, 5));
+    // One data missing
+    locations.add(ECStreamTestUtil.createIndexMap(1, 2, 4, 5));
+    // Two data missing including first
+    locations.add(ECStreamTestUtil.createIndexMap(2, 4, 5));
+    // One data and one parity missing
+    locations.add(ECStreamTestUtil.createIndexMap(2, 3, 4));
+    // No missing indexes
+    locations.add(ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+
+      OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+          stripeSize() * 3 + partialStripeSize, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+      dataGen = new SplittableRandom(randomSeed);
+      try (ECBlockReconstructedStripeInputStream ecb =
+          createInputStream(keyInfo)) {
+        // Read 3 full stripes
+        for (int i = 0; i < 3; i++) {
+          int read = ecb.readStripe(bufs);
+          for (int j = 0; j < bufs.length; j++) {
+            ECStreamTestUtil.assertBufferMatches(bufs[j], dataGen);
+          }
+          Assert.assertEquals(stripeSize(), read);
+
+          // Check the underlying streams have read 1 chunk per read:
+          for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+            Assert.assertEquals(chunkSize * (i + 1),
+                bis.getPos());
+          }
+          Assert.assertEquals(stripeSize() * (i + 1), ecb.getPos());
+          clearBuffers(bufs);
+        }
+        // The next read is a partial stripe
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(partialStripeSize, read);
+        ECStreamTestUtil.assertBufferMatches(bufs[0], dataGen);
+        ECStreamTestUtil.assertBufferMatches(bufs[1], dataGen);
+        Assert.assertEquals(0, bufs[2].remaining());
+        Assert.assertEquals(0, bufs[2].position());
+
+        // A further read should give EOF
+        clearBuffers(bufs);
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  @Test
+  public void testReadPartialStripe() throws IOException {
+    int blockLength = repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    ECStreamTestUtil
+        .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    dataGen = new SplittableRandom(randomSeed);
+    try (ECBlockReconstructedStripeInputStream ecb =
+        createInputStream(keyInfo)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      ECStreamTestUtil.assertBufferMatches(bufs[0], dataGen);
+      Assert.assertEquals(0, bufs[1].remaining());
+      Assert.assertEquals(0, bufs[1].position());
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 blockLength:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(blockLength, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeTwoChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 2 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    ECStreamTestUtil
+        .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    dataGen = new SplittableRandom(randomSeed);
+    try (ECBlockReconstructedStripeInputStream ecb =
+        createInputStream(keyInfo)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      ECStreamTestUtil.assertBufferMatches(bufs[0], dataGen);
+      ECStreamTestUtil.assertBufferMatches(bufs[1], dataGen);
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 chunk:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(chunkSize, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeThreeChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 3 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    ECStreamTestUtil
+        .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    // We have a length that is less than a stripe, so chunks 1 and 2 are full.
+    // Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(ECStreamTestUtil.createIndexMap(3, 4, 5));
+    // Two data missing
+    locations.add(ECStreamTestUtil.createIndexMap(1, 4, 5));
+    // One data missing - the last one
+    locations.add(ECStreamTestUtil.createIndexMap(1, 2, 5));
+    // One data and one parity missing
+    locations.add(ECStreamTestUtil.createIndexMap(2, 3, 4));
+    // One data and one parity missing
+    locations.add(ECStreamTestUtil.createIndexMap(1, 2, 4));
+    // No indexes missing
+    locations.add(ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+
+      OmKeyLocationInfo keyInfo =
+          ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+      dataGen = new SplittableRandom(randomSeed);
+      try (ECBlockReconstructedStripeInputStream ecb =
+          createInputStream(keyInfo)) {
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(blockLength, read);
+        ECStreamTestUtil.assertBufferMatches(bufs[0], dataGen);
+        ECStreamTestUtil.assertBufferMatches(bufs[1], dataGen);
+        ECStreamTestUtil.assertBufferMatches(bufs[2], dataGen);
+        // Check the underlying streams have been advanced by 1 chunk:
+        for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+          Assert.assertEquals(0, bis.getRemaining());
+        }
+        Assert.assertEquals(ecb.getPos(), blockLength);
+        clearBuffers(bufs);
+        // A further read should give EOF
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  @Test
+  public void testErrorThrownIfBlockNotLongEnough() throws IOException {
+    int blockLength = repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    ECStreamTestUtil
+        .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    // Set the parity buffer limit to be less than the block length
+    parity[0].limit(blockLength - 1);
+    parity[1].limit(blockLength - 1);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    try (ECBlockReconstructedStripeInputStream ecb =
+        createInputStream(keyInfo)) {
+      try {
+        ecb.readStripe(bufs);
+        Assert.fail("Read should have thrown an exception");
+      } catch (InsufficientLocationsException e) {
+        // expected
+      }
+    }
+  }
+
+  @Test
+  public void testSeek() throws IOException {
+    // Generate the input data for 3 full stripes and generate the parity
+    // and a partial stripe
+    int chunkSize = repConfig.getEcChunkSize();
+    int partialStripeSize = chunkSize * 2 - 1;
+    int dataLength = stripeSize() * 3 + partialStripeSize;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+    ECStreamTestUtil
+        .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, dataLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(ECStreamTestUtil.createIndexMap(1, 4, 5));
+    // One data missing
+    locations.add(ECStreamTestUtil.createIndexMap(1, 2, 4, 5));
+    // Two data missing including first
+    locations.add(ECStreamTestUtil.createIndexMap(2, 4, 5));
+    // One data and one parity missing
+    locations.add(ECStreamTestUtil.createIndexMap(2, 3, 4));
+    // No locations missing
+    locations.add(ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+
+      OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+          stripeSize() * 3 + partialStripeSize, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+      try (ECBlockReconstructedStripeInputStream ecb =
+          createInputStream(keyInfo)) {
+
+        // Read Stripe 1
+        int read = ecb.readStripe(bufs);
+        for (int j = 0; j < bufs.length; j++) {
+          validateContents(dataBufs[j], bufs[j], 0, chunkSize);
+        }
+        Assert.assertEquals(stripeSize(), read);
+        Assert.assertEquals(dataLength - stripeSize(), ecb.getRemaining());
+
+        // Seek to 0 and read again
+        clearBuffers(bufs);
+        ecb.seek(0);
+        ecb.readStripe(bufs);
+        for (int j = 0; j < bufs.length; j++) {
+          validateContents(dataBufs[j], bufs[j], 0, chunkSize);
+        }
+        Assert.assertEquals(stripeSize(), read);
+        Assert.assertEquals(dataLength - stripeSize(), ecb.getRemaining());
+
+        // Seek to the last stripe
+        // Seek to the last stripe
+        clearBuffers(bufs);
+        ecb.seek(stripeSize() * 3);
+        read = ecb.readStripe(bufs);
+        validateContents(dataBufs[0], bufs[0], 3 * chunkSize, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 3 * chunkSize, chunkSize - 1);
+        Assert.assertEquals(0, bufs[2].remaining());
+        Assert.assertEquals(partialStripeSize, read);
+        Assert.assertEquals(0, ecb.getRemaining());
+
+        // seek to the start of stripe 3
+        clearBuffers(bufs);
+        ecb.seek(stripeSize() * (long)2);
+        read = ecb.readStripe(bufs);
+        for (int j = 0; j < bufs.length; j++) {
+          validateContents(dataBufs[j], bufs[j], 2 * chunkSize, chunkSize);
+        }
+        Assert.assertEquals(stripeSize(), read);
+        Assert.assertEquals(partialStripeSize, ecb.getRemaining());
+      }
+    }
+  }
+
+  @Test
+  public void testSeekToPartialOffsetFails() {
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 4, 5);
+    OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+        stripeSize() * 3, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+    try (ECBlockReconstructedStripeInputStream ecb =
+        createInputStream(keyInfo)) {
+      try {
+        ecb.seek(10);
+        Assert.fail("Seek should have thrown an exception");
+      } catch (IOException e) {
+        Assert.assertEquals("Requested position 10 does not align " +
+            "with a stripe offset", e.getMessage());
+      }
+    }
+  }
+
+  private Integer getRandomStreamIndex(Set<Integer> set) {
+    return set.stream().skip(new Random().nextInt(set.size()))
+        .findFirst().orElse(null);
+  }
+
+  @Test
+  public void testErrorReadingBlockContinuesReading() throws IOException {
+    // Generate the input data for 3 full stripes and generate the parity.
+    int chunkSize = repConfig.getEcChunkSize();
+    int partialStripeSize = chunkSize * 2 - 1;
+    int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+        + partialStripeSize;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+        4 * chunkSize);
+    ECStreamTestUtil
+        .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    for (int k = 0; k < 5; k++) {
+      Set<Integer> failed = new HashSet<>();
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+
+      // Data block index 3 is missing and needs recovered initially.
+      Map<DatanodeDetails, Integer> dnMap =
+          ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+      OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+          stripeSize() * 3 + partialStripeSize, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+      try (ECBlockReconstructedStripeInputStream ecb =
+          createInputStream(keyInfo)) {
+        // After reading the first stripe, make one of the streams error
+        for (int i = 0; i < 3; i++) {
+          int read = ecb.readStripe(bufs);
+          for (int j = 0; j < bufs.length; j++) {
+            validateContents(dataBufs[j], bufs[j], i * chunkSize, chunkSize);
+          }
+          Assert.assertEquals(stripeSize() * (i + 1), ecb.getPos());
+          Assert.assertEquals(stripeSize(), read);
+          clearBuffers(bufs);
+          if (i == 0) {
+            Integer failStream =
+                getRandomStreamIndex(streamFactory.getStreamIndexes());
+            streamFactory.getBlockStream(failStream)
+                .setShouldError(true);
+            failed.add(failStream);
+          }
+        }
+        // The next read is a partial stripe
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(partialStripeSize, read);
+        validateContents(dataBufs[0], bufs[0], 3 * chunkSize, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 3 * chunkSize, chunkSize - 1);
+        Assert.assertEquals(0, bufs[2].remaining());
+        Assert.assertEquals(0, bufs[2].position());
+
+        // seek back to zero and read a stripe to re-open the streams
+        ecb.seek(0);
+        clearBuffers(bufs);
+        ecb.readStripe(bufs);
+        // Now fail another random stream and the read should fail with
+        // insufficient locations
+        Set<Integer> currentStreams =
+            new HashSet<>(streamFactory.getStreamIndexes());
+        currentStreams.removeAll(failed);
+        Integer failStream = getRandomStreamIndex(currentStreams);
+        streamFactory.getBlockStream(failStream)
+            .setShouldError(true);
+        try {
+          clearBuffers(bufs);
+          ecb.readStripe(bufs);
+          Assert.fail("InsufficientLocationsException expected");
+        } catch (InsufficientLocationsException e) {
+          // expected
+        }
+      }
+    }
+  }
+
+  @Test(expected = InsufficientLocationsException.class)
+  public void testAllLocationsFailOnFirstRead() throws IOException {
+    // This test simulates stale nodes. When the nodes are stale, but not yet
+    // dead, the locations will still be given to the client and it will try to
+    // read them, but the read will always fail.
+    // Additionally, if the key is small (less than 2 EC chunks), the locations
+    // for the indexes which are all padding will be returned to the client and
+    // this can confuse the "sufficient locations" check, resulting in a strange
+    // error when selecting parity indexes (HDDS-6258)
+    int chunkSize = repConfig.getEcChunkSize();
+    int partialStripeSize = chunkSize;
+    int blockLength = partialStripeSize;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), chunkSize);
+    ECStreamTestUtil
+        .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    streamFactory = new TestBlockInputStreamFactory();
+    addDataStreamsToFactory(dataBufs, parity);
+    // Fail all the indexes containing data on their first read.
+    streamFactory.setFailIndexes(indexesToList(1, 4, 5));
+    // The locations contain the padded indexes, as will often be the case
+    // when containers are reported by SCM.
+    Map<DatanodeDetails, Integer> dnMap =
+          ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+        blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    try (ECBlockReconstructedStripeInputStream ecb =
+             createInputStream(keyInfo)) {
+      ecb.readStripe(bufs);
+    }
+  }
+
+  @Test
+  public void testFailedLocationsAreNotRead() throws IOException {
+    // Generate the input data for 3 full stripes and generate the parity.
+    int chunkSize = repConfig.getEcChunkSize();
+    int partialStripeSize = chunkSize * 2 - 1;
+    int blockLength = chunkSize * repConfig.getData() * 3 + partialStripeSize;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+    ECStreamTestUtil.randomFill(dataBufs, chunkSize, dataGen, blockLength);
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    streamFactory = new TestBlockInputStreamFactory();
+    addDataStreamsToFactory(dataBufs, parity);
+
+    Map<DatanodeDetails, Integer> dnMap =
+        ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+    OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+        stripeSize() * 3 + partialStripeSize, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    dataGen = new SplittableRandom(randomSeed);
+    try (ECBlockReconstructedStripeInputStream ecb =
+        createInputStream(keyInfo)) {
+      List<DatanodeDetails> failed = new ArrayList<>();
+      // Set the first 3 DNs as failed
+      for (Map.Entry<DatanodeDetails, Integer> e : dnMap.entrySet()) {
+        if (e.getValue() <= 2) {
+          failed.add(e.getKey());
+        }
+      }
+      ecb.addFailedDatanodes(failed);
+
+      // Read full stripe
+      int read = ecb.readStripe(bufs);
+      for (int j = 0; j < bufs.length; j++) {
+        ECStreamTestUtil.assertBufferMatches(bufs[j], dataGen);
+      }
+      Assert.assertEquals(stripeSize(), read);
+
+      // Now ensure that streams with repIndexes 1, 2 and 3 have not been
+      // created in the stream factory, indicating we did not read them.
+      List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+      for (TestBlockInputStream stream : streams) {
+        Assert.assertTrue(stream.getEcReplicaIndex() > 2);
+      }
+    }
+  }
+
+  private ECBlockReconstructedStripeInputStream createInputStream(
+      OmKeyLocationInfo keyInfo) {
+    return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+        null, null, streamFactory, bufferPool, ecReconstructExecutor);
+  }
+
+  private List<Integer> indexesToList(int... indexes) {
+    List<Integer> list = new ArrayList<>();
+    for (int i : indexes) {
+      list.add(i);
+    }
+    return list;
+  }
+
+  private void addDataStreamsToFactory(ByteBuffer[] data, ByteBuffer[] parity) {
+    List<ByteBuffer> dataStreams = new ArrayList<>();
+    for (ByteBuffer b : data) {
+      dataStreams.add(b);
+    }
+    for (ByteBuffer b : parity) {
+      dataStreams.add(b);
+    }
+    streamFactory.setBlockStreamData(dataStreams);
+  }
+
+  /**
+   * Validates that the data buffer has the same contents as the source buffer,
+   * starting the checks in the src at offset and for count bytes.
+   * @param src The source of the data
+   * @param data The data which should be checked against the source
+   * @param offset The starting point in the src buffer
+   * @param count How many bytes to check.
+   */
+  private void validateContents(ByteBuffer src, ByteBuffer data, int offset,
+      int count) {
+    byte[] srcArray = src.array();
+    Assert.assertEquals(count, data.remaining());
+    for (int i = offset; i < offset + count; i++) {
+      Assert.assertEquals("Element " + i, srcArray[i], data.get());
+    }
+    data.flip();
+  }
+
+  /**
+   * Return a list of num ByteBuffers of the given size.
+   * @param num Number of buffers to create
+   * @param size The size of each buffer
+   * @return
+   */
+  private ByteBuffer[] allocateBuffers(int num, int size) {
+    ByteBuffer[] bufs = new ByteBuffer[num];
+    for (int i = 0; i < num; i++) {
+      bufs[i] = ByteBuffer.allocate(size);
+    }
+    return bufs;
+  }
+
+  private int stripeSize() {
+    return stripeSize(repConfig);
+  }
+
+  private int stripeSize(ECReplicationConfig rconfig) {
+    return rconfig.getEcChunkSize() * rconfig.getData();
+  }
+
+  private void clearBuffers(ByteBuffer[] bufs) {
+    for (ByteBuffer b : bufs) {
+      b.clear();
+    }
+  }
+
+  private ByteBuffer[] allocateByteBuffers(ECReplicationConfig rConfig) {
+    ByteBuffer[] bufs = new ByteBuffer[repConfig.getData()];
+    for (int i = 0; i < bufs.length; i++) {
+      bufs[i] = ByteBuffer.allocate(rConfig.getEcChunkSize());
+    }
+    return bufs;
+  }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
index 7f0ab38..07c2433 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
@@ -23,7 +23,8 @@
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.StorageUnit;
-import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
@@ -48,6 +49,7 @@
 import org.junit.runners.Parameterized;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
@@ -114,7 +116,7 @@
     conf.setFromObject(repConf);
 
     cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(4)
+        .setNumDatanodes(5)
         .setTotalPipelineNumLimit(5)
         .setBlockSize(BLOCK_SIZE)
         .setChunkSize(CHUNK_SIZE)
@@ -169,8 +171,14 @@
   }
 
   byte[] writeKey(String keyName, int dataLength) throws Exception {
+    ReplicationConfig repConfig = RatisReplicationConfig.getInstance(THREE);
+    return writeKey(keyName, repConfig, dataLength);
+  }
+
+  byte[] writeKey(String keyName, ReplicationConfig repConfig, int dataLength)
+      throws Exception {
     OzoneOutputStream key = TestHelper.createKey(keyName,
-        ReplicationType.RATIS, 0, objectStore, volumeName, bucketName);
+        repConfig, 0, objectStore, volumeName, bucketName);
 
     byte[] inputData = ContainerTestHelper.getFixedLengthString(
         keyString, dataLength).getBytes(UTF_8);
@@ -182,8 +190,15 @@
 
   byte[] writeRandomBytes(String keyName, int dataLength)
       throws Exception {
+    ReplicationConfig repConfig = RatisReplicationConfig.getInstance(THREE);
+    return writeRandomBytes(keyName, repConfig, dataLength);
+  }
+
+  byte[] writeRandomBytes(String keyName, ReplicationConfig repConfig,
+      int dataLength)
+      throws Exception {
     OzoneOutputStream key = TestHelper.createKey(keyName,
-        ReplicationType.RATIS, 0, objectStore, volumeName, bucketName);
+        repConfig, 0, objectStore, volumeName, bucketName);
 
     byte[] inputData = new byte[dataLength];
     RAND.nextBytes(inputData);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java
index 7f9dd70..442cbfe 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java
@@ -23,8 +23,10 @@
 import java.util.Arrays;
 
 import java.util.List;
+import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -33,6 +35,7 @@
 import org.apache.hadoop.hdds.scm.XceiverClientMetrics;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
 import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
 import org.apache.hadoop.hdds.scm.storage.ChunkInputStream;
 import org.apache.hadoop.ozone.client.io.KeyInputStream;
@@ -50,6 +53,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec.RS;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
 import static org.apache.hadoop.ozone.container.TestHelper.countReplicas;
 import static org.junit.Assert.fail;
@@ -95,6 +99,24 @@
   }
 
   /**
+   * This method does random seeks and reads and validates the reads are
+   * correct or not.
+   * @param dataLength
+   * @param keyInputStream
+   * @param inputData
+   * @param readSize
+   * @throws Exception
+   */
+  private void randomPositionSeek(int dataLength, KeyInputStream keyInputStream,
+      byte[] inputData, int readSize) throws Exception {
+    Random rand = new Random();
+    for (int i = 0; i < 100; i++) {
+      int position = rand.nextInt(dataLength - readSize);
+      validate(keyInputStream, inputData, position, readSize);
+    }
+  }
+
+  /**
    * This method seeks to specified seek value and read the data specified by
    * readLength and validate the read is correct or not.
    * @param keyInputStream
@@ -125,6 +147,7 @@
     testReadChunkWithByteArray();
     testReadChunkWithByteBuffer();
     testSkip();
+    testECSeek();
   }
 
   public void testInputStreams() throws Exception {
@@ -137,11 +160,13 @@
     // Verify BlockStreams and ChunkStreams
     int expectedNumBlockStreams = BufferUtils.getNumberOfBins(
         dataLength, BLOCK_SIZE);
-    List<BlockInputStream> blockStreams = keyInputStream.getBlockStreams();
+    List<BlockExtendedInputStream> blockStreams =
+        keyInputStream.getBlockStreams();
     Assert.assertEquals(expectedNumBlockStreams, blockStreams.size());
 
     int readBlockLength = 0;
-    for (BlockInputStream blockStream : blockStreams) {
+    for (BlockExtendedInputStream stream : blockStreams) {
+      BlockInputStream blockStream = (BlockInputStream) stream;
       int blockStreamLength = Math.min(BLOCK_SIZE,
           dataLength - readBlockLength);
       Assert.assertEquals(blockStreamLength, blockStream.getLength());
@@ -195,6 +220,36 @@
     keyInputStream.close();
   }
 
+  public void testECSeek() throws Exception {
+    int ecChunkSize = 1024 * 1024;
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS,
+        ecChunkSize);
+    String keyName = getNewKeyName();
+    // 3 full EC blocks plus one chunk
+    int dataLength = (9 * BLOCK_SIZE + ecChunkSize);
+
+    byte[] inputData = writeRandomBytes(keyName, repConfig, dataLength);
+    try (KeyInputStream keyInputStream = getKeyInputStream(keyName)) {
+
+      validate(keyInputStream, inputData, 0, ecChunkSize + 1234);
+
+      validate(keyInputStream, inputData, 200, ecChunkSize);
+
+      validate(keyInputStream, inputData, BLOCK_SIZE * 4, ecChunkSize);
+
+      validate(keyInputStream, inputData, BLOCK_SIZE * 4 + 200, ecChunkSize);
+
+      validate(keyInputStream, inputData, dataLength - ecChunkSize - 100,
+          ecChunkSize + 50);
+
+      randomPositionSeek(dataLength, keyInputStream, inputData,
+          ecChunkSize + 200);
+
+      // Read entire key.
+      validate(keyInputStream, inputData, 0, dataLength);
+    }
+  }
+
   public void testSeek() throws Exception {
     XceiverClientManager.resetXceiverClientMetrics();
     XceiverClientMetrics metrics = XceiverClientManager
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStreamEC.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStreamEC.java
new file mode 100644
index 0000000..fa6688f
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStreamEC.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.KeyInputStream;
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.MB;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test KeyInputStream with EC keys.
+ */
+public class TestKeyInputStreamEC {
+
+  @Test
+  public void testReadAgainstLargeBlockGroup() throws IOException {
+    int dataBlocks = 10;
+    int parityBlocks = 4;
+    ECReplicationConfig ec10And4RepConfig = new ECReplicationConfig(dataBlocks,
+        parityBlocks, ECReplicationConfig.EcCodec.RS, (int)(1 * MB));
+    // default blockSize of 256MB with EC 10+4 makes a large block group
+    long blockSize = 256 * MB;
+    long blockLength = dataBlocks * blockSize;
+    OmKeyInfo keyInfo = createOmKeyInfo(ec10And4RepConfig,
+        dataBlocks + parityBlocks, blockLength);
+
+    BlockExtendedInputStream blockInputStream =
+        new ECStreamTestUtil.TestBlockInputStream(new BlockID(1, 1),
+        blockLength, ByteBuffer.allocate(100));
+
+    BlockInputStreamFactory mockStreamFactory =
+        mock(BlockInputStreamFactory.class);
+    when(mockStreamFactory.create(any(), any(), any(), any(),
+        anyBoolean(), any(), any())).thenReturn(blockInputStream);
+
+    try (LengthInputStream kis = KeyInputStream.getFromOmKeyInfo(keyInfo,
+        null, true,  null, mockStreamFactory)) {
+      byte[] buf = new byte[100];
+      int readBytes = kis.read(buf, 0, 100);
+      Assert.assertEquals(100, readBytes);
+    }
+  }
+
+  private OmKeyInfo createOmKeyInfo(ReplicationConfig repConf,
+      int nodeCount, long blockLength) {
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+    for (int i = 0; i < nodeCount; i++) {
+      dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), i + 1);
+    }
+
+    Pipeline pipeline = Pipeline.newBuilder()
+        .setState(Pipeline.PipelineState.CLOSED)
+        .setId(PipelineID.randomId())
+        .setNodes(new ArrayList<>(dnMap.keySet()))
+        .setReplicaIndexes(dnMap)
+        .setReplicationConfig(repConf)
+        .build();
+
+    OmKeyLocationInfo blockInfo = new OmKeyLocationInfo.Builder()
+        .setBlockID(new BlockID(1, 1))
+        .setLength(blockLength)
+        .setOffset(0)
+        .setPipeline(pipeline)
+        .setPartNumber(0)
+        .build();
+
+    List<OmKeyLocationInfo> locations = new ArrayList<>();
+    locations.add(blockInfo);
+    return new OmKeyInfo.Builder()
+        .setBucketName("bucket")
+        .setVolumeName("volume")
+        .setDataSize(blockLength)
+        .setKeyName("someKey")
+        .setReplicationConfig(repConf)
+        .addOmKeyLocationInfoGroup(new OmKeyLocationInfoGroup(0, locations))
+        .build();
+  }
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java
index 7a4e1c3..dae6e38 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java
@@ -28,6 +28,8 @@
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -145,6 +147,13 @@
         .createKey(keyName, size, type, factor, new HashMap<>());
   }
 
+  public static OzoneOutputStream createKey(String keyName,
+      ReplicationConfig replicationConfig, long size, ObjectStore objectStore,
+      String volumeName, String bucketName) throws Exception {
+    return objectStore.getVolume(volumeName).getBucket(bucketName)
+        .createKey(keyName, size, replicationConfig, new HashMap<>());
+  }
+
   public static void validateData(String keyName, byte[] data,
       ObjectStore objectStore, String volumeName, String bucketName)
       throws Exception {
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
index d142384..b9eb231 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.ozone.freon;
 
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -27,6 +25,7 @@
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.junit.Assert;
 import org.junit.Test;
+import picocli.CommandLine;
 
 import java.time.Duration;
 
@@ -64,17 +63,19 @@
   }
 
   @Test
-  public void ratisTestLargeKey() throws Exception {
+  public void ratisTestLargeKey() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(1);
-    randomKeyGenerator.setNumOfBuckets(1);
-    randomKeyGenerator.setNumOfKeys(1);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setKeySize(20971520);
-    randomKeyGenerator.setValidateWrites(true);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "1",
+        "--key-size", "20971520",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
@@ -82,16 +83,18 @@
   }
 
   @Test
-  public void validateWriteTest() throws Exception {
+  public void validateWriteTest() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(2);
-    randomKeyGenerator.setNumOfBuckets(5);
-    randomKeyGenerator.setNumOfKeys(10);
-    randomKeyGenerator.setValidateWrites(true);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "2",
+        "--num-of-buckets", "5",
+        "--num-of-keys", "10",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
     Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithDatanodeFastRestart.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithDatanodeFastRestart.java
index 957c3d6..2c11804 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithDatanodeFastRestart.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithDatanodeFastRestart.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.ozone.freon;
 
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.container.TestHelper;
@@ -34,6 +32,7 @@
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
+import picocli.CommandLine;
 
 /**
  * Tests Freon with Datanode restarts without waiting for pipeline to close.
@@ -111,17 +110,19 @@
     startFreon();
   }
 
-  private void startFreon() throws Exception {
+  private void startFreon() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(1);
-    randomKeyGenerator.setNumOfBuckets(1);
-    randomKeyGenerator.setNumOfKeys(1);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setKeySize(20971520);
-    randomKeyGenerator.setValidateWrites(true);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "1",
+        "--key-size", "20971520",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithPipelineDestroy.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithPipelineDestroy.java
index 290eb5d..50f4971 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithPipelineDestroy.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithPipelineDestroy.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.ozone.freon;
 
 import org.apache.hadoop.hdds.HddsConfigKeys;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -36,6 +34,7 @@
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import picocli.CommandLine;
 
 import java.time.Duration;
 import java.util.concurrent.TimeUnit;
@@ -102,17 +101,19 @@
     startFreon();
   }
 
-  private void startFreon() throws Exception {
+  private void startFreon() {
     RandomKeyGenerator randomKeyGenerator =
         new RandomKeyGenerator(cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(1);
-    randomKeyGenerator.setNumOfBuckets(1);
-    randomKeyGenerator.setNumOfKeys(1);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setKeySize(20971520);
-    randomKeyGenerator.setValidateWrites(true);
-    randomKeyGenerator.call();
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "1",
+        "--key-size", "20971520",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
index cd6af31..132d086 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
@@ -20,8 +20,6 @@
 
 import java.time.Duration;
 
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
@@ -32,6 +30,7 @@
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
+import picocli.CommandLine;
 
 /**
  * Tests Freon, with MiniOzoneCluster.
@@ -77,67 +76,89 @@
   }
 
   @Test
-  public void defaultTest() throws Exception {
+  void testDefault() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(2);
-    randomKeyGenerator.setNumOfBuckets(5);
-    randomKeyGenerator.setNumOfKeys(10);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "2",
+        "--num-of-buckets", "5",
+        "--num-of-keys", "10");
+
     Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
   }
 
   @Test
-  public void multiThread() throws Exception {
+  void testECKey() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(10);
-    randomKeyGenerator.setNumOfBuckets(1);
-    randomKeyGenerator.setNumOfKeys(10);
-    randomKeyGenerator.setNumOfThreads(10);
-    randomKeyGenerator.setKeySize(10240);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "2",
+        "--num-of-buckets", "5",
+        "--num-of-keys", "10",
+        "--replication", "rs-3-2-1024k",
+        "--type", "EC"
+    );
+
+    Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
+    Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
+    Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
+  }
+
+  @Test
+  void testMultiThread() {
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "10",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "10",
+        "--num-of-threads", "10",
+        "--key-size", "10240",
+        "--factor", "THREE",
+        "--type", "RATIS"
+    );
+
     Assert.assertEquals(10, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
   }
 
   @Test
-  public void ratisTest3() throws Exception {
+  void testRatisKey() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(10);
-    randomKeyGenerator.setNumOfBuckets(1);
-    randomKeyGenerator.setNumOfKeys(10);
-    randomKeyGenerator.setNumOfThreads(10);
-    randomKeyGenerator.setKeySize(10240);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "10",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "10",
+        "--num-of-threads", "10",
+        "--key-size", "10240",
+        "--factor", "THREE",
+        "--type", "RATIS"
+    );
+
     Assert.assertEquals(10, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
   }
 
   @Test
-  public void bigFileThan2GB() throws Exception {
+  void testKeyLargerThan2GB() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(1);
-    randomKeyGenerator.setNumOfBuckets(1);
-    randomKeyGenerator.setNumOfKeys(1);
-    randomKeyGenerator.setNumOfThreads(1);
-    randomKeyGenerator.setKeySize(10L + Integer.MAX_VALUE);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.setValidateWrites(true);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "1",
+        "--num-of-threads", "1",
+        "--key-size", String.valueOf(10L + Integer.MAX_VALUE),
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
@@ -145,18 +166,20 @@
   }
 
   @Test
-  public void fileWithSizeZero() throws Exception {
+  void testZeroSizeKey() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(1);
-    randomKeyGenerator.setNumOfBuckets(1);
-    randomKeyGenerator.setNumOfKeys(1);
-    randomKeyGenerator.setNumOfThreads(1);
-    randomKeyGenerator.setKeySize(0);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.setValidateWrites(true);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "1",
+        "--num-of-threads", "1",
+        "--key-size", "0",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
@@ -164,33 +187,37 @@
   }
 
   @Test
-  public void testThreadPoolSize() throws Exception {
+  void testThreadPoolSize() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(1);
-    randomKeyGenerator.setNumOfBuckets(1);
-    randomKeyGenerator.setNumOfKeys(1);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.setNumOfThreads(10);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "1",
+        "--num-of-threads", "10",
+        "--factor", "THREE",
+        "--type", "RATIS"
+    );
+
     Assert.assertEquals(10, randomKeyGenerator.getThreadPoolSize());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
   }
 
   @Test
   @Flaky("HDDS-5993")
-  public void cleanObjectsTest() throws Exception {
+  void cleanObjectsTest() {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
-    randomKeyGenerator.setNumOfVolumes(2);
-    randomKeyGenerator.setNumOfBuckets(5);
-    randomKeyGenerator.setNumOfKeys(10);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
-    randomKeyGenerator.setNumOfThreads(10);
-    randomKeyGenerator.setCleanObjects(true);
-    randomKeyGenerator.call();
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "2",
+        "--num-of-buckets", "5",
+        "--num-of-keys", "10",
+        "--num-of-threads", "10",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--clean-objects"
+    );
+
     Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
     Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
index 9b9d6d8..c9babb8 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
@@ -34,6 +34,10 @@
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.client.ContainerBlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.HddsWhiteboxTestUtils;
 import org.apache.hadoop.hdds.scm.pipeline.MockPipeline;
 import org.apache.hadoop.hdds.utils.db.Table;
@@ -251,7 +255,7 @@
   @Test
   public void testKeyOps() throws Exception {
     // This test needs a cluster with DNs and SCM to wait on safemode
-    clusterBuilder.setNumDatanodes(3);
+    clusterBuilder.setNumDatanodes(5);
     conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED, true);
     startCluster();
     String volumeName = UUID.randomUUID().toString();
@@ -260,7 +264,8 @@
         .getInternalState(ozoneManager, "keyManager");
     KeyManager mockKm = Mockito.spy(keyManager);
     TestDataUtil.createVolumeAndBucket(cluster, volumeName, bucketName);
-    OmKeyArgs keyArgs = createKeyArgs(volumeName, bucketName);
+    OmKeyArgs keyArgs = createKeyArgs(volumeName, bucketName,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
     doKeyOps(keyArgs);
 
     MetricsRecordBuilder omMetrics = getMetrics("OMMetrics");
@@ -273,18 +278,26 @@
     assertCounter("NumKeys", 0L, omMetrics);
     assertCounter("NumInitiateMultipartUploads", 1L, omMetrics);
 
-    keyArgs = createKeyArgs(volumeName, bucketName);
+    keyArgs = createKeyArgs(volumeName, bucketName,
+        new ECReplicationConfig("rs-3-2-1024K"));
+    doKeyOps(keyArgs);
+    omMetrics = getMetrics("OMMetrics");
+    assertCounter("NumKeyOps", 14L, omMetrics);
+
+    keyArgs = createKeyArgs(volumeName, bucketName,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
     OpenKeySession keySession = writeClient.openKey(keyArgs);
     writeClient.commitKey(keyArgs, keySession.getId());
-    keyArgs = createKeyArgs(volumeName, bucketName);
+    keyArgs = createKeyArgs(volumeName, bucketName,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
     keySession = writeClient.openKey(keyArgs);
     writeClient.commitKey(keyArgs, keySession.getId());
-    keyArgs = createKeyArgs(volumeName, bucketName);
+    keyArgs = createKeyArgs(volumeName, bucketName,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
     keySession = writeClient.openKey(keyArgs);
     writeClient.commitKey(keyArgs, keySession.getId());
     writeClient.deleteKey(keyArgs);
 
-
     omMetrics = getMetrics("OMMetrics");
     assertCounter("NumKeys", 2L, omMetrics);
 
@@ -299,17 +312,18 @@
 
     // inject exception to test for Failure Metrics on the write path
     mockWritePathExceptions(OmBucketInfo.class);
-    keyArgs = createKeyArgs(volumeName, bucketName);
+    keyArgs = createKeyArgs(volumeName, bucketName,
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
     doKeyOps(keyArgs);
 
     omMetrics = getMetrics("OMMetrics");
-    assertCounter("NumKeyOps", 21L, omMetrics);
-    assertCounter("NumKeyAllocate", 5L, omMetrics);
-    assertCounter("NumKeyLookup", 2L, omMetrics);
-    assertCounter("NumKeyDeletes", 3L, omMetrics);
-    assertCounter("NumKeyLists", 2L, omMetrics);
-    assertCounter("NumTrashKeyLists", 2L, omMetrics);
-    assertCounter("NumInitiateMultipartUploads", 2L, omMetrics);
+    assertCounter("NumKeyOps", 28L, omMetrics);
+    assertCounter("NumKeyAllocate", 6L, omMetrics);
+    assertCounter("NumKeyLookup", 3L, omMetrics);
+    assertCounter("NumKeyDeletes", 4L, omMetrics);
+    assertCounter("NumKeyLists", 3L, omMetrics);
+    assertCounter("NumTrashKeyLists", 3L, omMetrics);
+    assertCounter("NumInitiateMultipartUploads", 3L, omMetrics);
 
     assertCounter("NumKeyAllocateFails", 1L, omMetrics);
     assertCounter("NumKeyLookupFails", 1L, omMetrics);
@@ -554,8 +568,8 @@
     }
   }
 
-  private OmKeyArgs createKeyArgs(String volumeName, String bucketName)
-      throws IOException {
+  private OmKeyArgs createKeyArgs(String volumeName, String bucketName,
+      ReplicationConfig repConfig) throws IOException {
     OmKeyLocationInfo keyLocationInfo = new OmKeyLocationInfo.Builder()
         .setBlockID(new BlockID(new ContainerBlockID(1, 1)))
         .setPipeline(MockPipeline.createSingleNodePipeline())
@@ -569,8 +583,10 @@
         .setBucketName(bucketName)
         .setKeyName(keyName)
         .setAcls(Lists.emptyList())
+        .setReplicationConfig(repConfig)
         .build();
   }
+
   private OmVolumeArgs createVolumeArgs() {
     String volumeName = UUID.randomUUID().toString();
     return new OmVolumeArgs.Builder()
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
index c0f6342..a09af96 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
@@ -35,12 +35,19 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.OzoneAdmin;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.OFSPath;
 import org.apache.hadoop.fs.ozone.OzoneFsShell;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl;
 import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
+import org.apache.hadoop.ozone.client.io.KeyOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.ha.ConfUtils;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OzoneManager;
@@ -101,8 +108,8 @@
   private static String testFilePathString;
   private static OzoneConfiguration conf = null;
   private static MiniOzoneCluster cluster = null;
-  private static OzoneShell ozoneShell = null;
-  private static OzoneAdmin ozoneAdminShell = null;
+  private OzoneShell ozoneShell = null;
+  private OzoneAdmin ozoneAdminShell = null;
 
   private final ByteArrayOutputStream out = new ByteArrayOutputStream();
   private final ByteArrayOutputStream err = new ByteArrayOutputStream();
@@ -134,19 +141,18 @@
     testFile.getParentFile().mkdirs();
     testFile.createNewFile();
 
-    ozoneShell = new OzoneShell();
-    ozoneAdminShell = new OzoneAdmin();
-
     // Init HA cluster
     omServiceId = "om-service-test1";
     numOfOMs = 3;
     clusterId = UUID.randomUUID().toString();
     scmId = UUID.randomUUID().toString();
+    final int numDNs = 5;
     cluster = MiniOzoneCluster.newOMHABuilder(conf)
         .setClusterId(clusterId)
         .setScmId(scmId)
         .setOMServiceId(omServiceId)
         .setNumOfOzoneManagers(numOfOMs)
+        .setNumDatanodes(numDNs)
         .build();
     cluster.waitForClusterToBeReady();
   }
@@ -167,6 +173,8 @@
 
   @Before
   public void setup() throws UnsupportedEncodingException {
+    ozoneShell = new OzoneShell();
+    ozoneAdminShell = new OzoneAdmin();
     System.setOut(new PrintStream(out, false, DEFAULT_ENCODING));
     System.setErr(new PrintStream(err, false, DEFAULT_ENCODING));
   }
@@ -499,17 +507,17 @@
     execute(ozoneAdminShell, args);
 
     // Test case 3: list THREE replica container
-    String factor = "--factor=THREE";
+    String factor = "--replication=THREE";
     args = new String[] {"container", "list", "--scm",
         "localhost:" + cluster.getStorageContainerManager().getClientRpcPort(),
-        factor};
+        factor, "--type=RATIS"};
     execute(ozoneAdminShell, args);
 
     // Test case 4: list ONE replica container
-    factor = "--factor=ONE";
+    factor = "--replication=ONE";
     args = new String[] {"container", "list", "--scm",
         "localhost:" + cluster.getStorageContainerManager().getClientRpcPort(),
-        factor};
+        factor, "--type=RATIS"};
     execute(ozoneAdminShell, args);
   }
 
@@ -893,6 +901,120 @@
   }
 
   @Test
+  public void testCreateBucketWithECReplicationConfig() throws Exception {
+    final String volumeName = "volume100";
+    getVolume(volumeName);
+    String[] args =
+        new String[] {"bucket", "create", "/volume100/bucket0", "-t", "EC",
+            "-r", "rs-3-2-1024k"};
+    execute(ozoneShell, args);
+
+    OzoneVolume volume =
+        cluster.getClient().getObjectStore().getVolume(volumeName);
+    OzoneBucket bucket = volume.getBucket("bucket0");
+    try (OzoneOutputStream out = bucket.createKey("myKey", 2000)) {
+      Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+    }
+  }
+
+  @Test
+  public void testPutKeyOnBucketWithECReplicationConfig() throws Exception {
+    final String volumeName = UUID.randomUUID().toString();
+    final String bucketName = UUID.randomUUID().toString();
+    final String keyName = UUID.randomUUID().toString();
+    getVolume(volumeName);
+    String bucketPath =
+        Path.SEPARATOR + volumeName + Path.SEPARATOR + bucketName;
+    String[] args =
+        new String[] {"bucket", "create", bucketPath, "-t", "EC", "-r",
+            "rs-3-2-1024k"};
+    execute(ozoneShell, args);
+
+    args = new String[] {"key", "put", bucketPath + Path.SEPARATOR + keyName,
+        testFilePathString};
+    execute(ozoneShell, args);
+
+    OzoneKeyDetails key =
+        cluster.getClient().getObjectStore().getVolume(volumeName)
+            .getBucket(bucketName).getKey(keyName);
+    assertEquals(HddsProtos.ReplicationType.EC,
+        key.getReplicationConfig().getReplicationType());
+  }
+
+  @Test
+  public void testCreateBucketWithRatisReplicationConfig() throws Exception {
+    final String volumeName = "volume101";
+    getVolume(volumeName);
+    String[] args =
+        new String[] {"bucket", "create", "/volume101/bucket1", "-t", "RATIS",
+            "-r", "3"};
+    execute(ozoneShell, args);
+
+    OzoneVolume volume =
+        cluster.getClient().getObjectStore().getVolume(volumeName);
+    OzoneBucket bucket = volume.getBucket("bucket1");
+    try (OzoneOutputStream out = bucket.createKey("myKey", 2000)) {
+      Assert.assertTrue(out.getOutputStream() instanceof KeyOutputStream);
+      Assert.assertFalse(out.getOutputStream() instanceof ECKeyOutputStream);
+    }
+  }
+
+  @Test
+  public void testSetECReplicationConfigOnBucket() throws Exception {
+    final String volumeName = "volume110";
+    getVolume(volumeName);
+    String bucketPath = "/volume110/bucket0";
+    String[] args = new String[] {"bucket", "create", bucketPath};
+    execute(ozoneShell, args);
+
+    OzoneVolume volume =
+        cluster.getClient().getObjectStore().getVolume(volumeName);
+    OzoneBucket bucket = volume.getBucket("bucket0");
+    try (OzoneOutputStream out = bucket.createKey("myNonECKey", 1024)) {
+      Assert.assertFalse(out.getOutputStream().getClass().getName()
+          .equals(ECKeyOutputStream.class.getName()));
+    }
+
+    args = new String[] {"bucket", "set-replication-config", bucketPath, "-t",
+        "EC", "-r", "rs-3-2-1024k"};
+    execute(ozoneShell, args);
+    bucket = volume.getBucket("bucket0");
+    try (OzoneOutputStream out = bucket.createKey("newECKey", 1024)) {
+      Assert.assertTrue(out.getOutputStream().getClass().getName()
+          .equals(ECKeyOutputStream.class.getName()));
+    }
+
+    args = new String[] {"bucket", "set-replication-config", bucketPath, "-t",
+        "STAND_ALONE", "-r", "ONE"};
+    execute(ozoneShell, args);
+    bucket = volume.getBucket("bucket0");
+    try (OzoneOutputStream out = bucket.createKey("newNonECKey", 1024)) {
+      Assert.assertFalse(out.getOutputStream().getClass().getName()
+          .equals(ECKeyOutputStream.class.getName()));
+    }
+  }
+
+  @Test
+  public void testCreateBucketWithECReplicationConfigWithoutReplicationParam() {
+    getVolume("volume102");
+    String[] args =
+        new String[] {"bucket", "create", "/volume102/bucket2", "-t", "EC"};
+    try {
+      execute(ozoneShell, args);
+      Assert.fail("Must throw Exception when missing replication param");
+    } catch (Exception e) {
+      Assert.assertEquals(
+          "Replication can't be null. Replication type passed was : EC",
+          e.getCause().getMessage());
+    }
+  }
+
+  private void getVolume(String volumeName) {
+    String[] args = new String[] {"volume", "create",
+        "o3://" + omServiceId + "/" + volumeName};
+    execute(ozoneShell, args);
+  }
+
   public void testListVolumeBucketKeyShouldPrintValidJsonArray()
       throws UnsupportedEncodingException {
 
diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
index 45a191d..1bad215 100644
--- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
+++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
@@ -356,6 +356,7 @@
     LAYOUT_FEATURE_FINALIZATION_FAILED = 72;
     PREPARE_FAILED = 73;
     NOT_SUPPORTED_OPERATION_WHEN_PREPARED = 74;
+    NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION = 75;
 }
 
 /**
@@ -546,6 +547,7 @@
     optional uint64 usedNamespace = 17;
     optional BucketLayoutProto bucketLayout = 18;
     optional string owner = 19;
+    optional hadoop.hdds.DefaultReplicationConfig defaultReplicationConfig = 20;
 }
 
 enum StorageTypeProto {
@@ -624,6 +626,7 @@
     optional uint64 quotaInBytes = 8;
     optional uint64 quotaInNamespace = 9;
     optional string ownerName = 10;
+    optional hadoop.hdds.DefaultReplicationConfig defaultReplicationConfig = 11;
 }
 
 message PrefixInfo {
@@ -760,8 +763,8 @@
     required string bucketName = 2;
     required string keyName = 3;
     optional uint64 dataSize = 4;
-    optional hadoop.hdds.ReplicationType type = 5;
-    optional hadoop.hdds.ReplicationFactor factor = 6;
+    optional hadoop.hdds.ReplicationType type = 5 [default = NONE];
+    optional hadoop.hdds.ReplicationFactor factor = 6 [default = ZERO];
     repeated KeyLocation keyLocations = 7;
     optional bool isMultipartKey = 8;
     optional string multipartUploadID = 9;
@@ -783,6 +786,7 @@
 
     // When it is a head operation which is to check whether key exist
     optional bool headOp = 18;
+    optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 19;
 }
 
 message KeyLocation {
@@ -816,7 +820,7 @@
     required string keyName = 3;
     required uint64 dataSize = 4;
     required hadoop.hdds.ReplicationType type = 5;
-    required hadoop.hdds.ReplicationFactor factor = 6;
+    optional hadoop.hdds.ReplicationFactor factor = 6;
     repeated KeyLocationList keyLocationList = 7;
     required uint64 creationTime = 8;
     required uint64 modificationTime = 9;
@@ -827,6 +831,7 @@
     optional uint64 objectID = 14;
     optional uint64 updateID = 15;
     optional uint64 parentID = 16;
+    optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 17;
 }
 
 message DirectoryInfo {
@@ -1211,11 +1216,12 @@
     required string uploadID = 1;
     required uint64 creationTime = 2;
     required hadoop.hdds.ReplicationType type = 3;
-    required hadoop.hdds.ReplicationFactor factor = 4;
+    optional hadoop.hdds.ReplicationFactor factor = 4;
     repeated PartKeyInfo partKeyInfoList = 5;
     optional uint64 objectID = 6;
     optional uint64 updateID = 7;
     optional uint64 parentID = 8;
+    optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 9;
 }
 
 message PartKeyInfo {
@@ -1273,7 +1279,7 @@
     optional uint32 nextPartNumberMarker = 4;
     optional bool isTruncated = 5;
     repeated PartInfo partsList = 6;
-
+    optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 7;
 }
 
 message ListMultipartUploadsRequest {
@@ -1294,8 +1300,8 @@
     required string uploadId = 4;
     required uint64 creationTime = 5;
     required hadoop.hdds.ReplicationType type = 6;
-    required hadoop.hdds.ReplicationFactor factor = 7;
-
+    optional hadoop.hdds.ReplicationFactor factor = 7;
+    optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 8;
 }
 
 message PartInfo {
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index 124b272..cd04acb 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -711,9 +711,10 @@
             omPartInfoList.add(omPartInfo);
 
             //if there are parts, use replication type from one of the parts
-            replicationConfig = ReplicationConfig.fromProtoTypeAndFactor(
-                    partKeyInfo.getPartKeyInfo().getType(),
-                    partKeyInfo.getPartKeyInfo().getFactor());
+            replicationConfig = ReplicationConfig.fromProto(
+                partKeyInfo.getPartKeyInfo().getType(),
+                partKeyInfo.getPartKeyInfo().getFactor(),
+                partKeyInfo.getPartKeyInfo().getEcReplicationConfig());
             count++;
           }
         }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneConfigUtil.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneConfigUtil.java
new file mode 100644
index 0000000..ea559f8
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneConfigUtil.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
+/**
+ * Utility class for ozone configurations.
+ */
+public final class OzoneConfigUtil {
+  private OzoneConfigUtil() {
+  }
+
+  public static ReplicationConfig resolveReplicationConfigPreference(
+      HddsProtos.ReplicationType clientType,
+      HddsProtos.ReplicationFactor clientFactor,
+      HddsProtos.ECReplicationConfig clientECReplicationConfig,
+      DefaultReplicationConfig bucketDefaultReplicationConfig,
+      ReplicationConfig omDefaultReplicationConfig) {
+    ReplicationConfig replicationConfig = null;
+    if (clientType != HddsProtos.ReplicationType.NONE) {
+      // Client passed the replication config, so let's use it.
+      replicationConfig = ReplicationConfig
+          .fromProto(clientType, clientFactor, clientECReplicationConfig);
+    } else {
+      // type is NONE, so, let's look for the bucket defaults.
+      if (bucketDefaultReplicationConfig != null) {
+        boolean hasECReplicationConfig = bucketDefaultReplicationConfig
+            .getType() == ReplicationType.EC && bucketDefaultReplicationConfig
+            .getEcReplicationConfig() != null;
+        // Since Bucket defaults are available, let's inherit
+        replicationConfig = ReplicationConfig.fromProto(
+            ReplicationType.toProto(bucketDefaultReplicationConfig.getType()),
+            ReplicationFactor
+                .toProto(bucketDefaultReplicationConfig.getFactor()),
+            hasECReplicationConfig ?
+                bucketDefaultReplicationConfig.getEcReplicationConfig()
+                    .toProto() :
+                null);
+      } else {
+        // if bucket defaults also not available, then use server defaults.
+        replicationConfig = omDefaultReplicationConfig;
+      }
+    }
+    return replicationConfig;
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 1a0e7d4..7731b9e 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -60,6 +60,8 @@
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.ConfigurationException;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -237,6 +239,10 @@
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_VOLUME_LISTALL_ALLOWED_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_KEY;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_TYPE_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_TYPE_KEY;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.DETECTED_LOOP_IN_BUCKET_LINKS;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_AUTH_METHOD;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_REQUEST;
@@ -2610,7 +2616,9 @@
     auditMap.put(OzoneConsts.BUCKET, bucket);
     try {
       metrics.incNumBucketInfos();
-      return bucketManager.getBucketInfo(volume, bucket);
+      final OmBucketInfo bucketInfo =
+          bucketManager.getBucketInfo(volume, bucket);
+      return bucketInfo;
     } catch (Exception ex) {
       metrics.incNumBucketInfoFails();
       auditSuccess = false;
@@ -3675,6 +3683,17 @@
         OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT);
   }
 
+  public ReplicationConfig getDefaultReplicationConfig() {
+    final String replication = configuration.getTrimmed(
+        OZONE_SERVER_DEFAULT_REPLICATION_KEY,
+        OZONE_SERVER_DEFAULT_REPLICATION_DEFAULT);
+    final String type = configuration.getTrimmed(
+        OZONE_SERVER_DEFAULT_REPLICATION_TYPE_KEY,
+        OZONE_SERVER_DEFAULT_REPLICATION_TYPE_DEFAULT);
+    return ReplicationConfig.parse(ReplicationType.valueOf(type),
+        replication, configuration);
+  }
+
   public String getOMDefaultBucketLayout() {
     return this.defaultBucketLayout;
   }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
index 7644746..37d3e71 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
@@ -41,8 +41,13 @@
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
 import org.apache.hadoop.ozone.om.response.bucket.OMBucketCreateResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketEncryptionInfoProto;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
@@ -51,6 +56,7 @@
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateBucketResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.ozone.security.acl.OzoneObj;
@@ -365,4 +371,26 @@
 
   }
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.CreateBucket
+  )
+  public static OMRequest disallowCreateBucketWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager()
+        .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getCreateBucketRequest()
+          .getBucketInfo().hasDefaultReplicationConfig()
+          && req.getCreateBucketRequest().getBucketInfo()
+          .getDefaultReplicationConfig().hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
index 17c4e39..d237c1c 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
@@ -23,12 +23,19 @@
 
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.audit.AuditLogger;
 import org.apache.hadoop.ozone.audit.OMAction;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -182,6 +189,13 @@
             dbBucketInfo.getQuotaInNamespace());
       }
 
+      DefaultReplicationConfig defaultReplicationConfig =
+          omBucketArgs.getDefaultReplicationConfig();
+      if (defaultReplicationConfig != null) {
+        // Resetting the default replication config.
+        bucketInfoBuilder.setDefaultReplicationConfig(defaultReplicationConfig);
+      }
+
       bucketInfoBuilder.setCreationTime(dbBucketInfo.getCreationTime());
       bucketInfoBuilder.setModificationTime(
           setBucketPropertyRequest.getModificationTime());
@@ -295,4 +309,28 @@
     return true;
   }
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.SetBucketProperty
+  )
+  public static OMRequest disallowSetBucketPropertyWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager()
+        .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      SetBucketPropertyRequest propReq =
+          req.getSetBucketPropertyRequest();
+      if (propReq.hasBucketArgs()
+          && propReq.getBucketArgs().hasDefaultReplicationConfig()
+          && propReq.getBucketArgs().getDefaultReplicationConfig()
+          .hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
index 223986b..62e8bc2 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
@@ -28,7 +28,9 @@
 
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
@@ -37,6 +39,12 @@
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.ozone.security.acl.OzoneObj;
 import org.slf4j.Logger;
@@ -196,9 +204,9 @@
         long baseObjId = ozoneManager.getObjectIdFromTxId(trxnLogIndex);
         List<OzoneAcl> inheritAcls = omPathInfo.getAcls();
 
-        dirKeyInfo = createDirectoryKeyInfoWithACL(keyName,
-            keyArgs, baseObjId,
-            OzoneAclUtil.fromProtobuf(keyArgs.getAclsList()), trxnLogIndex);
+        dirKeyInfo = createDirectoryKeyInfoWithACL(keyName, keyArgs, baseObjId,
+            OzoneAclUtil.fromProtobuf(keyArgs.getAclsList()), trxnLogIndex,
+            ozoneManager.getDefaultReplicationConfig());
 
         missingParentInfos = getAllParentInfo(ozoneManager, keyArgs,
             missingParents, inheritAcls, trxnLogIndex);
@@ -278,8 +286,10 @@
 
       LOG.debug("missing parent {} getting added to KeyTable", missingKey);
       // what about keyArgs for parent directories? TODO
-      OmKeyInfo parentKeyInfo = createDirectoryKeyInfoWithACL(
-          missingKey, keyArgs, nextObjId, inheritAcls, trxnLogIndex);
+      OmKeyInfo parentKeyInfo =
+          createDirectoryKeyInfoWithACL(missingKey, keyArgs, nextObjId,
+              inheritAcls, trxnLogIndex,
+              ozoneManager.getDefaultReplicationConfig());
       objectCount++;
 
       missingParentInfos.add(parentKeyInfo);
@@ -336,35 +346,73 @@
    * @param keyArgs
    * @param objectId
    * @param transactionIndex
+   * @param serverDefaultReplConfig
    * @return the OmKeyInfo structure
    */
-  public static OmKeyInfo createDirectoryKeyInfoWithACL(
-      String keyName, KeyArgs keyArgs, long objectId,
-      List<OzoneAcl> inheritAcls, long transactionIndex) {
-    return dirKeyInfoBuilderNoACL(keyName, keyArgs, objectId)
-        .setAcls(inheritAcls).setUpdateID(transactionIndex).build();
+  public static OmKeyInfo createDirectoryKeyInfoWithACL(String keyName,
+      KeyArgs keyArgs, long objectId, List<OzoneAcl> inheritAcls,
+      long transactionIndex, ReplicationConfig serverDefaultReplConfig) {
+    return dirKeyInfoBuilderNoACL(keyName, keyArgs, objectId,
+        serverDefaultReplConfig).setAcls(inheritAcls)
+        .setUpdateID(transactionIndex).build();
   }
 
   private static OmKeyInfo.Builder dirKeyInfoBuilderNoACL(String keyName,
-      KeyArgs keyArgs, long objectId) {
+      KeyArgs keyArgs, long objectId,
+      ReplicationConfig serverDefaultReplConfig) {
     String dirName = OzoneFSUtils.addTrailingSlashIfNeeded(keyName);
 
-    return new OmKeyInfo.Builder()
-        .setVolumeName(keyArgs.getVolumeName())
-        .setBucketName(keyArgs.getBucketName())
-        .setKeyName(dirName)
-        .setOmKeyLocationInfos(Collections.singletonList(
-            new OmKeyLocationInfoGroup(0, new ArrayList<>())))
-        .setCreationTime(keyArgs.getModificationTime())
-        .setModificationTime(keyArgs.getModificationTime())
-        .setDataSize(0)
-        .setReplicationConfig(ReplicationConfig
-                .fromProtoTypeAndFactor(keyArgs.getType(), keyArgs.getFactor()))
-        .setObjectID(objectId)
-        .setUpdateID(objectId);
+    OmKeyInfo.Builder keyInfoBuilder =
+        new OmKeyInfo.Builder()
+            .setVolumeName(keyArgs.getVolumeName())
+            .setBucketName(keyArgs.getBucketName())
+            .setKeyName(dirName)
+            .setOmKeyLocationInfos(Collections.singletonList(
+                new OmKeyLocationInfoGroup(0, new ArrayList<>())))
+            .setCreationTime(keyArgs.getModificationTime())
+            .setModificationTime(keyArgs.getModificationTime())
+            .setDataSize(0);
+    if (keyArgs.getFactor() != null && keyArgs
+        .getFactor() != HddsProtos.ReplicationFactor.ZERO && keyArgs
+        .getType() != HddsProtos.ReplicationType.EC) {
+      // Factor available and not an EC replication config.
+      keyInfoBuilder.setReplicationConfig(ReplicationConfig
+          .fromProtoTypeAndFactor(keyArgs.getType(), keyArgs.getFactor()));
+    } else if (keyArgs.getType() == HddsProtos.ReplicationType.EC) {
+      // Found EC type
+      keyInfoBuilder.setReplicationConfig(
+          new ECReplicationConfig(keyArgs.getEcReplicationConfig()));
+    } else {
+      // default type
+      keyInfoBuilder.setReplicationConfig(serverDefaultReplConfig);
+    }
+
+    keyInfoBuilder.setObjectID(objectId);
+    return keyInfoBuilder;
   }
 
   static long getMaxNumOfRecursiveDirs() {
     return MAX_NUM_OF_RECURSIVE_DIRS;
   }
+
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.CreateDirectory
+  )
+  public static OMRequest disallowCreateDirectoryWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager().
+        isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getCreateDirectoryRequest().getKeyArgs()
+          .hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java
index 0b4f943..2199691 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java
@@ -28,13 +28,20 @@
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
 import org.apache.hadoop.ozone.om.response.file.OMFileCreateResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -113,26 +120,23 @@
     final long requestedSize = keyArgs.getDataSize() > 0 ?
         keyArgs.getDataSize() : scmBlockSize;
 
-    boolean useRatis = ozoneManager.shouldUseRatis();
-
     HddsProtos.ReplicationFactor factor = keyArgs.getFactor();
-    if (factor == null) {
-      factor = useRatis ? HddsProtos.ReplicationFactor.THREE :
-          HddsProtos.ReplicationFactor.ONE;
-    }
-
     HddsProtos.ReplicationType type = keyArgs.getType();
-    if (type == null) {
-      type = useRatis ? HddsProtos.ReplicationType.RATIS :
-          HddsProtos.ReplicationType.STAND_ALONE;
-    }
+
+    final OmBucketInfo bucketInfo = ozoneManager
+        .getBucketInfo(keyArgs.getVolumeName(), keyArgs.getBucketName());
+    final ReplicationConfig repConfig = OzoneConfigUtil
+        .resolveReplicationConfigPreference(type, factor,
+            keyArgs.getEcReplicationConfig(),
+            bucketInfo.getDefaultReplicationConfig(),
+            ozoneManager.getDefaultReplicationConfig());
 
     // TODO: Here we are allocating block with out any check for
     //  bucket/key/volume or not and also with out any authorization checks.
 
     List< OmKeyLocationInfo > omKeyLocationInfoList =
         allocateBlock(ozoneManager.getScmClient(),
-              ozoneManager.getBlockTokenSecretManager(), type, factor,
+              ozoneManager.getBlockTokenSecretManager(), repConfig,
               new ExcludeList(), requestedSize, scmBlockSize,
               ozoneManager.getPreallocateBlocksMax(),
               ozoneManager.isGrpcBlockTokenEnabled(),
@@ -239,14 +243,19 @@
       }
 
       // do open key
-      OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(
-          omMetadataManager.getBucketKey(volumeName, bucketName));
+      omBucketInfo =
+          getBucketInfo(omMetadataManager, volumeName, bucketName);
+      final ReplicationConfig repConfig = OzoneConfigUtil
+          .resolveReplicationConfigPreference(keyArgs.getType(),
+              keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+              omBucketInfo.getDefaultReplicationConfig(),
+              ozoneManager.getDefaultReplicationConfig());
 
       omKeyInfo = prepareKeyInfo(omMetadataManager, keyArgs, dbKeyInfo,
           keyArgs.getDataSize(), locations, getFileEncryptionInfo(keyArgs),
-          ozoneManager.getPrefixManager(), bucketInfo, trxnLogIndex,
+          ozoneManager.getPrefixManager(), omBucketInfo, trxnLogIndex,
           ozoneManager.getObjectIdFromTxId(trxnLogIndex),
-          ozoneManager.isRatisEnabled());
+          ozoneManager.isRatisEnabled(), repConfig);
 
       long openVersion = omKeyInfo.getLatestVersionLocations().getVersion();
       long clientID = createFileRequest.getClientID();
@@ -262,8 +271,6 @@
           .stream().map(OmKeyLocationInfo::getFromProtobuf)
           .collect(Collectors.toList());
       omKeyInfo.appendNewBlocks(newLocationList, false);
-
-      omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
       // check bucket and volume quota
       long preAllocatedSpace = newLocationList.size()
           * ozoneManager.getScmBlockSize()
@@ -385,4 +392,24 @@
           OMException.ResultCodes.DIRECTORY_NOT_FOUND);
     }
   }
+
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = CreateFile
+  )
+  public static OMRequest disallowCreateFileWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager()
+        .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getCreateFileRequest().getKeyArgs().hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestWithFSO.java
index 762163b..9a74dfa 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestWithFSO.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestWithFSO.java
@@ -19,9 +19,11 @@
 package org.apache.hadoop.ozone.om.request.file;
 
 import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.ozone.audit.OMAction;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
@@ -162,13 +164,18 @@
       // do open key
       OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(
           omMetadataManager.getBucketKey(volumeName, bucketName));
+      final ReplicationConfig repConfig = OzoneConfigUtil
+          .resolveReplicationConfigPreference(keyArgs.getType(),
+              keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+              bucketInfo.getDefaultReplicationConfig(),
+              ozoneManager.getDefaultReplicationConfig());
 
       OmKeyInfo omFileInfo = prepareFileInfo(omMetadataManager, keyArgs,
               dbFileInfo, keyArgs.getDataSize(), locations,
               getFileEncryptionInfo(keyArgs), ozoneManager.getPrefixManager(),
               bucketInfo, pathInfoFSO, trxnLogIndex,
               pathInfoFSO.getLeafNodeObjectId(),
-              ozoneManager.isRatisEnabled());
+              ozoneManager.isRatisEnabled(), repConfig);
 
       long openVersion = omFileInfo.getLatestVersionLocations().getVersion();
       long clientID = createFileRequest.getClientID();
@@ -184,9 +191,9 @@
 
       omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
       // check bucket and volume quota
-      long preAllocatedSpace = newLocationList.size()
-              * ozoneManager.getScmBlockSize()
-              * omFileInfo.getReplicationConfig().getRequiredNodes();
+      long preAllocatedSpace =
+          newLocationList.size() * ozoneManager.getScmBlockSize() * repConfig
+              .getRequiredNodes();
       checkBucketQuotaInBytes(omBucketInfo, preAllocatedSpace);
       checkBucketQuotaInNamespace(omBucketInfo, 1L);
 
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
index 39f090b..c151bcb 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
@@ -25,10 +25,17 @@
 
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
@@ -99,14 +106,15 @@
     //  one uses direct omclient we might be in trouble.
 
 
+    ReplicationConfig repConfig = ReplicationConfig.fromProto(keyArgs.getType(),
+        keyArgs.getFactor(), keyArgs.getEcReplicationConfig());
     // To allocate atleast one block passing requested size and scmBlockSize
     // as same value. When allocating block requested size is same as
     // scmBlockSize.
     List<OmKeyLocationInfo> omKeyLocationInfoList =
         allocateBlock(ozoneManager.getScmClient(),
-            ozoneManager.getBlockTokenSecretManager(), keyArgs.getType(),
-            keyArgs.getFactor(), excludeList, ozoneManager.getScmBlockSize(),
-            ozoneManager.getScmBlockSize(),
+            ozoneManager.getBlockTokenSecretManager(), repConfig, excludeList,
+            ozoneManager.getScmBlockSize(), ozoneManager.getScmBlockSize(),
             ozoneManager.getPreallocateBlocksMax(),
             ozoneManager.isGrpcBlockTokenEnabled(), ozoneManager.getOMNodeId());
 
@@ -224,8 +232,9 @@
       omBucketInfo.incrUsedBytes(preAllocatedSpace);
       omResponse.setAllocateBlockResponse(AllocateBlockResponse.newBuilder()
           .setKeyLocation(blockLocation).build());
+      OmBucketInfo shortBucketInfo = omBucketInfo.copyObject();
       omClientResponse = new OMAllocateBlockResponse(omResponse.build(),
-          openKeyInfo, clientID, omBucketInfo.copyObject(), getBucketLayout());
+          openKeyInfo, clientID, shortBucketInfo, getBucketLayout());
 
       LOG.debug("Allocated block for Volume:{}, Bucket:{}, OpenKey:{}",
           volumeName, bucketName, openKeyName);
@@ -250,4 +259,24 @@
 
     return omClientResponse;
   }
+
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.AllocateBlock
+  )
+  public static OMRequest disallowAllocateBlockWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager()
+        .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getAllocateBlockRequest().getKeyArgs().hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
index 3195e92..047b5f3 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
@@ -38,6 +38,12 @@
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -241,12 +247,11 @@
       // AllocatedBlock. The space occupied by the Key shall be based on
       // the actual Key size, and the total Block size applied before should
       // be subtracted.
-      long correctedSpace = omKeyInfo.getDataSize() * factor -
+      long correctedSpace = omKeyInfo.getReplicatedSize() -
           allocatedLocationInfoList.size() * scmBlockSize * factor;
       // Subtract the size of blocks to be overwritten.
       if (keyToDelete != null) {
-        correctedSpace -= keyToDelete.getDataSize() *
-            keyToDelete.getReplicationConfig().getRequiredNodes();
+        correctedSpace -= keyToDelete.getReplicatedSize();
       }
 
       omBucketInfo.incrUsedBytes(correctedSpace);
@@ -322,4 +327,24 @@
               commitKeyRequest);
     }
   }
+
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.CommitKey
+  )
+  public static OMRequest disallowCommitKeyWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager()
+        .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getCommitKeyRequest().getKeyArgs().hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
index 32dbe04..5e2ef2d 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
@@ -179,12 +179,11 @@
       // AllocatedBlock. The space occupied by the Key shall be based on
       // the actual Key size, and the total Block size applied before should
       // be subtracted.
-      long correctedSpace = omKeyInfo.getDataSize() * factor -
+      long correctedSpace = omKeyInfo.getReplicatedSize() -
           allocatedLocationInfoList.size() * scmBlockSize * factor;
       // Subtract the size of blocks to be overwritten.
       if (keyToDelete != null) {
-        correctedSpace -= keyToDelete.getDataSize() *
-            keyToDelete.getReplicationConfig().getRequiredNodes();
+        correctedSpace -= keyToDelete.getReplicatedSize();
       }
       omBucketInfo.incrUsedBytes(correctedSpace);
 
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
index 6e41211..d45b2b2 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
@@ -27,14 +27,21 @@
 
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.hadoop.ozone.om.request.file.OMDirectoryCreateRequest;
 import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -121,19 +128,16 @@
       final long requestedSize = keyArgs.getDataSize() > 0 ?
           keyArgs.getDataSize() : scmBlockSize;
 
-      boolean useRatis = ozoneManager.shouldUseRatis();
-
       HddsProtos.ReplicationFactor factor = keyArgs.getFactor();
-      if (factor == null) {
-        factor = useRatis ? HddsProtos.ReplicationFactor.THREE :
-            HddsProtos.ReplicationFactor.ONE;
-      }
-
       HddsProtos.ReplicationType type = keyArgs.getType();
-      if (type == null) {
-        type = useRatis ? HddsProtos.ReplicationType.RATIS :
-            HddsProtos.ReplicationType.STAND_ALONE;
-      }
+
+      final OmBucketInfo bucketInfo = ozoneManager
+          .getBucketInfo(keyArgs.getVolumeName(), keyArgs.getBucketName());
+      final ReplicationConfig repConfig = OzoneConfigUtil
+          .resolveReplicationConfigPreference(type, factor,
+              keyArgs.getEcReplicationConfig(),
+              bucketInfo.getDefaultReplicationConfig(),
+              ozoneManager.getDefaultReplicationConfig());
 
       // TODO: Here we are allocating block with out any check for
       //  bucket/key/volume or not and also with out any authorization checks.
@@ -142,7 +146,7 @@
 
       List<OmKeyLocationInfo> omKeyLocationInfoList =
           allocateBlock(ozoneManager.getScmClient(),
-              ozoneManager.getBlockTokenSecretManager(), type, factor,
+              ozoneManager.getBlockTokenSecretManager(), repConfig,
               new ExcludeList(), requestedSize, scmBlockSize,
               ozoneManager.getPreallocateBlocksMax(),
               ozoneManager.isGrpcBlockTokenEnabled(),
@@ -264,11 +268,17 @@
         numMissingParents = missingParentInfos.size();
       }
 
+      ReplicationConfig replicationConfig = OzoneConfigUtil
+          .resolveReplicationConfigPreference(keyArgs.getType(),
+              keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+              bucketInfo.getDefaultReplicationConfig(),
+              ozoneManager.getDefaultReplicationConfig());
+
       omKeyInfo = prepareKeyInfo(omMetadataManager, keyArgs, dbKeyInfo,
           keyArgs.getDataSize(), locations, getFileEncryptionInfo(keyArgs),
           ozoneManager.getPrefixManager(), bucketInfo, trxnLogIndex,
           ozoneManager.getObjectIdFromTxId(trxnLogIndex),
-          ozoneManager.isRatisEnabled());
+          ozoneManager.isRatisEnabled(), replicationConfig);
 
       long openVersion = omKeyInfo.getLatestVersionLocations().getVersion();
       long clientID = createKeyRequest.getClientID();
@@ -291,7 +301,7 @@
       // commitKey.
       long preAllocatedSpace = newLocationList.size()
           * ozoneManager.getScmBlockSize()
-          * omKeyInfo.getReplicationConfig().getRequiredNodes();
+          * replicationConfig.getRequiredNodes();
       // check bucket and volume quota
       checkBucketQuotaInBytes(omBucketInfo, preAllocatedSpace);
       checkBucketQuotaInNamespace(omBucketInfo, 1L);
@@ -369,4 +379,24 @@
           createKeyRequest);
     }
   }
+
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.CreateKey
+  )
+  public static OMRequest disallowCreateKeyWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager()
+        .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getCreateKeyRequest().getKeyArgs().hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequestWithFSO.java
index 7bd9f86..ed6ca16 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequestWithFSO.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequestWithFSO.java
@@ -19,9 +19,11 @@
 package org.apache.hadoop.ozone.om.request.key;
 
 import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.ozone.audit.OMAction;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
@@ -146,13 +148,18 @@
       // do open key
       OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(
               omMetadataManager.getBucketKey(volumeName, bucketName));
+      final ReplicationConfig repConfig = OzoneConfigUtil
+          .resolveReplicationConfigPreference(keyArgs.getType(),
+              keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+              bucketInfo.getDefaultReplicationConfig(),
+              ozoneManager.getDefaultReplicationConfig());
 
       OmKeyInfo omFileInfo = prepareFileInfo(omMetadataManager, keyArgs,
               dbFileInfo, keyArgs.getDataSize(), locations,
               getFileEncryptionInfo(keyArgs), ozoneManager.getPrefixManager(),
               bucketInfo, pathInfoFSO, trxnLogIndex,
               pathInfoFSO.getLeafNodeObjectId(),
-              ozoneManager.isRatisEnabled());
+              ozoneManager.isRatisEnabled(), repConfig);
 
       long openVersion = omFileInfo.getLatestVersionLocations().getVersion();
       long clientID = createKeyRequest.getClientID();
@@ -168,9 +175,9 @@
 
       omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
       // check bucket and volume quota
-      long preAllocatedSpace = newLocationList.size()
-              * ozoneManager.getScmBlockSize()
-              * omFileInfo.getReplicationConfig().getRequiredNodes();
+      long preAllocatedSpace =
+          newLocationList.size() * ozoneManager.getScmBlockSize() * repConfig
+              .getRequiredNodes();
       checkBucketQuotaInBytes(omBucketInfo, preAllocatedSpace);
       checkBucketQuotaInNamespace(omBucketInfo, 1L);
 
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java
index 26a68b8..8a90246 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java
@@ -32,6 +32,7 @@
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
 import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
@@ -41,6 +42,7 @@
 import org.apache.hadoop.ozone.om.PrefixManager;
 import org.apache.hadoop.ozone.om.ResolvedBucket;
 import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.hadoop.ozone.om.helpers.KeyValueUtil;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
@@ -49,7 +51,7 @@
 import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
-import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.QuotaUtil;
 import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
 import org.apache.hadoop.ozone.om.request.OMClientRequestUtils;
 import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
@@ -63,7 +65,6 @@
     .EncryptedKeyVersion;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
@@ -130,23 +131,21 @@
   @SuppressWarnings("parameternumber")
   protected List< OmKeyLocationInfo > allocateBlock(ScmClient scmClient,
       OzoneBlockTokenSecretManager secretManager,
-      HddsProtos.ReplicationType replicationType,
-      HddsProtos.ReplicationFactor replicationFactor,
-      ExcludeList excludeList, long requestedSize, long scmBlockSize,
-      int preallocateBlocksMax, boolean grpcBlockTokenEnabled, String omID)
-      throws IOException {
-
-    int numBlocks = Math.min((int) ((requestedSize - 1) / scmBlockSize + 1),
-        preallocateBlocksMax);
+      ReplicationConfig replicationConfig, ExcludeList excludeList,
+      long requestedSize, long scmBlockSize, int preallocateBlocksMax,
+      boolean grpcBlockTokenEnabled, String omID) throws IOException {
+    int dataGroupSize = replicationConfig instanceof ECReplicationConfig
+        ? ((ECReplicationConfig) replicationConfig).getData() : 1;
+    int numBlocks = (int) Math.min(preallocateBlocksMax,
+        (requestedSize - 1) / (scmBlockSize * dataGroupSize) + 1);
 
     List<OmKeyLocationInfo> locationInfos = new ArrayList<>(numBlocks);
     String remoteUser = getRemoteUser().getShortUserName();
     List<AllocatedBlock> allocatedBlocks;
     try {
-      allocatedBlocks = scmClient.getBlockClient().allocateBlock(scmBlockSize,
-          numBlocks, ReplicationConfig.fromProtoTypeAndFactor(replicationType,
-              replicationFactor),
-          omID, excludeList);
+      allocatedBlocks = scmClient.getBlockClient()
+          .allocateBlock(scmBlockSize, numBlocks, replicationConfig, omID,
+              excludeList);
     } catch (SCMException ex) {
       if (ex.getResult()
           .equals(SCMException.ResultCodes.SAFE_MODE_EXCEPTION)) {
@@ -579,11 +578,10 @@
    */
   protected static long sumBlockLengths(OmKeyInfo omKeyInfo) {
     long bytesUsed = 0;
-    int keyFactor = omKeyInfo.getReplicationConfig().getRequiredNodes();
-
     for (OmKeyLocationInfoGroup group: omKeyInfo.getKeyLocationVersions()) {
       for (OmKeyLocationInfo locationInfo : group.getLocationList()) {
-        bytesUsed += locationInfo.getLength() * keyFactor;
+        bytesUsed += QuotaUtil.getReplicatedSize(
+            locationInfo.getLength(), omKeyInfo.getReplicationConfig());
       }
     }
 
@@ -617,12 +615,13 @@
           @Nullable FileEncryptionInfo encInfo,
           @Nonnull PrefixManager prefixManager,
           @Nullable OmBucketInfo omBucketInfo,
-          long transactionLogIndex, long objectID, boolean isRatisEnabled)
+          long transactionLogIndex, long objectID, boolean isRatisEnabled,
+          ReplicationConfig replicationConfig)
           throws IOException {
 
     return prepareFileInfo(omMetadataManager, keyArgs, dbKeyInfo, size,
             locations, encInfo, prefixManager, omBucketInfo, null,
-            transactionLogIndex, objectID, isRatisEnabled);
+            transactionLogIndex, objectID, isRatisEnabled, replicationConfig);
   }
 
   /**
@@ -640,7 +639,7 @@
           @Nullable OmBucketInfo omBucketInfo,
           OMFileRequest.OMPathInfoWithFSO omPathInfo,
           long transactionLogIndex, long objectID,
-          boolean isRatisEnabled)
+          boolean isRatisEnabled, ReplicationConfig replicationConfig)
           throws IOException {
     if (keyArgs.getIsMultipartKey()) {
       return prepareMultipartFileInfo(omMetadataManager, keyArgs,
@@ -663,14 +662,13 @@
       // modification time.
       dbKeyInfo.setModificationTime(keyArgs.getModificationTime());
       dbKeyInfo.setUpdateID(transactionLogIndex, isRatisEnabled);
+      dbKeyInfo.setReplicationConfig(replicationConfig);
       return dbKeyInfo;
     }
 
     // the key does not exist, create a new object.
     // Blocks will be appended as version 0.
-    return createFileInfo(keyArgs, locations,
-            ReplicationConfig.fromProtoTypeAndFactor(
-                    keyArgs.getType(), keyArgs.getFactor()),
+    return createFileInfo(keyArgs, locations, replicationConfig,
             keyArgs.getDataSize(), encInfo, prefixManager,
             omBucketInfo, omPathInfo, transactionLogIndex, objectID);
   }
@@ -689,7 +687,6 @@
       @Nullable OmBucketInfo omBucketInfo,
       OMFileRequest.OMPathInfoWithFSO omPathInfo,
       long transactionLogIndex, long objectID) {
-
     OmKeyInfo.Builder builder = new OmKeyInfo.Builder();
     builder.setVolumeName(keyArgs.getVolumeName())
             .setBucketName(keyArgs.getBucketName())
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequest.java
index c54faf6..4aaec04 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequest.java
@@ -23,7 +23,9 @@
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.ozone.audit.OMAction;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
@@ -32,13 +34,19 @@
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.request.key.OMKeyRequest;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
 import org.apache.hadoop.ozone.om.response.s3.multipart.S3InitiateMultipartUploadResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartInfoInitiateRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartInfoInitiateResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.ozone.security.acl.OzoneObj;
@@ -168,10 +176,14 @@
       // care of in the final complete multipart upload. AWS S3 behavior is
       // also like this, even when key exists in a bucket, user can still
       // initiate MPU.
-
-      final ReplicationConfig replicationConfig =
-          ReplicationConfig.fromProtoTypeAndFactor(
-              keyArgs.getType(), keyArgs.getFactor());
+      final OmBucketInfo bucketInfo = omMetadataManager.getBucketTable()
+          .get(omMetadataManager.getBucketKey(volumeName, bucketName));
+      final ReplicationConfig replicationConfig = OzoneConfigUtil
+          .resolveReplicationConfigPreference(keyArgs.getType(),
+              keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+              bucketInfo != null ?
+                  bucketInfo.getDefaultReplicationConfig() :
+                  null, ozoneManager.getDefaultReplicationConfig());
 
       multipartKeyInfo = new OmMultipartKeyInfo.Builder()
           .setUploadID(keyArgs.getMultipartUploadID())
@@ -182,9 +194,6 @@
           .setUpdateID(transactionLogIndex)
           .build();
 
-      OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(
-          omMetadataManager.getBucketKey(volumeName, bucketName));
-
       omKeyInfo = new OmKeyInfo.Builder()
           .setVolumeName(volumeName)
           .setBucketName(bucketName)
@@ -267,4 +276,26 @@
           multipartInfoInitiateRequest);
     }
   }
+
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.InitiateMultiPartUpload
+  )
+  public static OMRequest
+      disallowInitiateMultiPartUploadWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager()
+        .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getInitiateMultiPartUploadRequest().getKeyArgs()
+          .hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequestWithFSO.java
index f1abc21..3f73ffb 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequestWithFSO.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequestWithFSO.java
@@ -24,8 +24,10 @@
 import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
 import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
@@ -151,9 +153,14 @@
       // care of in the final complete multipart upload. AWS S3 behavior is
       // also like this, even when key exists in a bucket, user can still
       // initiate MPU.
-      final ReplicationConfig replicationConfig =
-              ReplicationConfig.fromProtoTypeAndFactor(
-                      keyArgs.getType(), keyArgs.getFactor());
+      final OmBucketInfo bucketInfo = omMetadataManager.getBucketTable()
+          .get(omMetadataManager.getBucketKey(volumeName, bucketName));
+      final ReplicationConfig replicationConfig = OzoneConfigUtil
+          .resolveReplicationConfigPreference(keyArgs.getType(),
+              keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+              bucketInfo != null ?
+                  bucketInfo.getDefaultReplicationConfig() :
+                  null, ozoneManager.getDefaultReplicationConfig());
 
       multipartKeyInfo = new OmMultipartKeyInfo.Builder()
           .setUploadID(keyArgs.getMultipartUploadID())
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequest.java
index f524275..0959c71 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequest.java
@@ -25,8 +25,15 @@
 import com.google.common.base.Optional;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.QuotaUtil;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.ozone.security.acl.OzoneObj;
 import org.slf4j.Logger;
@@ -163,14 +170,14 @@
       // When abort uploaded key, we need to subtract the PartKey length from
       // the volume usedBytes.
       long quotaReleased = 0;
-      int keyFactor = omKeyInfo.getReplicationConfig().getRequiredNodes();
       Iterator iter =
           multipartKeyInfo.getPartKeyInfoMap().entrySet().iterator();
       while (iter.hasNext()) {
         Map.Entry entry = (Map.Entry)iter.next();
         PartKeyInfo iterPartKeyInfo = (PartKeyInfo)entry.getValue();
-        quotaReleased +=
-            iterPartKeyInfo.getPartKeyInfo().getDataSize() * keyFactor;
+        quotaReleased += QuotaUtil.getReplicatedSize(
+            iterPartKeyInfo.getPartKeyInfo().getDataSize(),
+            omKeyInfo.getReplicationConfig());
       }
       omBucketInfo.incrUsedBytes(-quotaReleased);
 
@@ -254,4 +261,25 @@
         volumeName, bucketName, keyName, multipartUploadID);
     return multipartKey;
   }
+
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.AbortMultiPartUpload
+  )
+  public static OMRequest disallowAbortMultiPartUploadWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager().isAllowed(
+        OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getAbortMultiPartUploadRequest().getKeyArgs()
+          .hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequest.java
index 517a46f..901f583 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequest.java
@@ -33,9 +33,14 @@
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.request.key.OMKeyRequest;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
 import org.apache.hadoop.ozone.om.response.s3.multipart
     .S3MultipartUploadCommitPartResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .KeyArgs;
@@ -47,6 +52,7 @@
     .OMRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.ozone.security.acl.OzoneObj;
 import org.apache.hadoop.util.Time;
@@ -227,7 +233,7 @@
       // AllocatedBlock. The space occupied by the Key shall be based on
       // the actual Key size, and the total Block size applied before should
       // be subtracted.
-      long correctedSpace = omKeyInfo.getDataSize() * factor -
+      long correctedSpace = omKeyInfo.getReplicatedSize() -
           keyArgs.getKeyLocationsList().size() * scmBlockSize * factor;
       omBucketInfo.incrUsedBytes(correctedSpace);
 
@@ -334,5 +340,25 @@
     return omMetadataManager.getMultipartKey(volumeName, bucketName,
         keyName, uploadID);
   }
-}
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.CommitMultiPartUpload
+  )
+  public static OMRequest disallowCommitMultiPartUploadWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager().isAllowed(
+        OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getCommitMultiPartUploadRequest().getKeyArgs()
+          .hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
index 3111a1c..3f205a8 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
@@ -30,7 +30,6 @@
 import java.util.TreeMap;
 
 import org.apache.hadoop.hdds.client.ReplicationConfig;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
 import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -49,8 +48,13 @@
 import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
 import org.apache.hadoop.ozone.om.request.key.OMKeyRequest;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
 import org.apache.hadoop.ozone.om.response.s3.multipart.S3MultipartUploadCompleteResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadCompleteRequest;
@@ -58,6 +62,7 @@
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.ozone.security.acl.OzoneObj;
 import org.apache.hadoop.util.Time;
@@ -352,10 +357,8 @@
       String ozoneKey, TreeMap<Integer, PartKeyInfo> partKeyInfoMap,
       List<OmKeyLocationInfo> partLocationInfos, long dataSize)
           throws IOException {
-    HddsProtos.ReplicationType type = partKeyInfoMap.lastEntry().getValue()
-        .getPartKeyInfo().getType();
-    HddsProtos.ReplicationFactor factor =
-        partKeyInfoMap.lastEntry().getValue().getPartKeyInfo().getFactor();
+    OzoneManagerProtocolProtos.KeyInfo partKeyInfo =
+        partKeyInfoMap.lastEntry().getValue().getPartKeyInfo();
 
     OmKeyInfo omKeyInfo = getOmKeyInfoFromKeyTable(ozoneKey, keyName,
             omMetadataManager);
@@ -372,8 +375,9 @@
       OmKeyInfo.Builder builder =
           new OmKeyInfo.Builder().setVolumeName(volumeName)
           .setBucketName(bucketName).setKeyName(dbOpenKeyInfo.getKeyName())
-          .setReplicationConfig(
-              ReplicationConfig.fromProtoTypeAndFactor(type, factor))
+          .setReplicationConfig(ReplicationConfig.fromProto(
+              partKeyInfo.getType(), partKeyInfo.getFactor(),
+              partKeyInfo.getEcReplicationConfig()))
           .setCreationTime(keyArgs.getModificationTime())
           .setModificationTime(keyArgs.getModificationTime())
           .setDataSize(dataSize)
@@ -567,5 +571,26 @@
           new CacheValue<>(Optional.of(omBucketInfo), transactionLogIndex));
     }
   }
-}
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.CompleteMultiPartUpload
+  )
+  public static OMRequest
+      disallowCompleteMultiPartUploadWithECReplicationConfig(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager().isAllowed(
+        OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+      if (req.getCompleteMultiPartUploadRequest().getKeyArgs()
+          .hasEcReplicationConfig()) {
+        throw new OMException("Cluster does not have the Erasure Coded"
+            + " Storage support feature finalized yet, but the request contains"
+            + " an Erasure Coded replication type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestValidations.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestValidations.java
index fe34b74..e2e127a 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestValidations.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestValidations.java
@@ -17,6 +17,7 @@
 package org.apache.hadoop.ozone.om.request.validation;
 
 import com.google.protobuf.ServiceException;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
 import org.slf4j.Logger;
@@ -59,20 +60,26 @@
     return this;
   }
 
-  public OMRequest validateRequest(OMRequest request) throws ServiceException {
+  public OMRequest validateRequest(OMRequest request)
+      throws Exception {
     List<Method> validations = registry.validationsFor(
         conditions(request), request.getCmdType(), PRE_PROCESS);
 
     OMRequest validatedRequest = request.toBuilder().build();
     try {
       for (Method m : validations) {
-        validatedRequest =
-            (OMRequest) m.invoke(null, validatedRequest, context);
         LOG.debug("Running the {} request pre-process validation from {}.{}",
             m.getName(), m.getDeclaringClass().getPackage().getName(),
             m.getDeclaringClass().getSimpleName());
+        validatedRequest =
+            (OMRequest) m.invoke(null, validatedRequest, context);
       }
-    } catch (IllegalAccessException | InvocationTargetException e) {
+    } catch (InvocationTargetException e) {
+      if (e.getCause() instanceof OMException) {
+        throw (OMException) e.getCause();
+      }
+      throw new ServiceException(e);
+    } catch (IllegalAccessException e) {
       throw new ServiceException(e);
     }
     return validatedRequest;
@@ -86,11 +93,11 @@
     OMResponse validatedResponse = response.toBuilder().build();
     try {
       for (Method m : validations) {
-        validatedResponse =
-            (OMResponse) m.invoke(null, request, response, context);
         LOG.debug("Running the {} request post-process validation from {}.{}",
             m.getName(), m.getDeclaringClass().getPackage().getName(),
             m.getDeclaringClass().getSimpleName());
+        validatedResponse =
+            (OMResponse) m.invoke(null, request, response, context);
       }
     } catch (InvocationTargetException | IllegalAccessException e) {
       throw new ServiceException(e);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
index 5338c16..aa639b2 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
@@ -29,7 +29,10 @@
  */
 public enum OMLayoutFeature implements LayoutFeature {
   //////////////////////////////  //////////////////////////////
-  INITIAL_VERSION(0, "Initial Layout Version");
+  INITIAL_VERSION(0, "Initial Layout Version"),
+
+  ERASURE_CODED_STORAGE_SUPPORT(1, "Ozone version with built in support for"
+      + " Erasure Coded block data storage.");
 
 
   ///////////////////////////////  /////////////////////////////
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
index a6fe773..cb1eadb 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.ozone.om.upgrade;
 
-import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION;
 
 import java.io.IOException;
 import java.lang.reflect.Method;
@@ -83,7 +83,7 @@
           layoutFeature.name(),
           layoutFeature.layoutVersion(),
           lvm.getMetadataLayoutVersion()),
-          NOT_SUPPORTED_OPERATION);
+          NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
     }
   }
 
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
index 06aa6c5..a2efa6f 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
@@ -30,6 +30,7 @@
 import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException;
 import org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
@@ -131,7 +132,15 @@
   @Override
   public OMResponse submitRequest(RpcController controller,
       OMRequest request) throws ServiceException {
-    OMRequest validatedRequest = requestValidations.validateRequest(request);
+    OMRequest validatedRequest;
+    try {
+      validatedRequest = requestValidations.validateRequest(request);
+    } catch (Exception e) {
+      if (e instanceof OMException) {
+        return createErrorResponse(request, (OMException) e);
+      }
+      throw new ServiceException(e);
+    }
 
     OMResponse response = 
         dispatcher.processRequest(validatedRequest, this::processRequest,
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
index fd98078..ecec945 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
@@ -22,6 +22,9 @@
 import java.util.List;
 import java.util.stream.Collectors;
 
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.UpgradeFinalizationStatus;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.utils.db.SequenceNumberNotFoundException;
@@ -45,6 +48,10 @@
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CheckVolumeAccessRequest;
@@ -58,6 +65,7 @@
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoVolumeResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListBucketsRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListBucketsResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListKeysRequest;
@@ -72,7 +80,9 @@
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadListPartsResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneFileStatusProto;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PrepareStatusResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RepeatedKeyInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
@@ -393,10 +403,36 @@
 
     resp.setKeyInfo(keyInfo.getProtobuf(keyArgs.getHeadOp(), clientVersion));
 
-
     return resp.build();
   }
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+      processingPhase = RequestProcessingPhase.POST_PROCESS,
+      requestType = Type.LookupKey
+  )
+  public static OMResponse disallowLookupKeyResponseWithECReplicationConfig(
+      OMRequest req, OMResponse resp, ValidationContext ctx)
+      throws ServiceException {
+    if (!resp.hasLookupKeyResponse()) {
+      return resp;
+    }
+    if (resp.getLookupKeyResponse().getKeyInfo().hasEcReplicationConfig()) {
+      resp = resp.toBuilder()
+          .setStatus(Status.NOT_SUPPORTED_OPERATION)
+          .setMessage("Key is a key with Erasure Coded replication, which"
+              + " the client can not understand.\n"
+              + "Please upgrade the client before trying to read the key: "
+              + req.getLookupKeyRequest().getKeyArgs().getVolumeName()
+              + "/" + req.getLookupKeyRequest().getKeyArgs().getBucketName()
+              + "/" + req.getLookupKeyRequest().getKeyArgs().getKeyName()
+              + ".")
+          .clearLookupKeyResponse()
+          .build();
+    }
+    return resp;
+  }
+
   private ListBucketsResponse listBuckets(ListBucketsRequest request)
       throws IOException {
     ListBucketsResponse.Builder resp =
@@ -432,6 +468,33 @@
     return resp.build();
   }
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+      processingPhase = RequestProcessingPhase.POST_PROCESS,
+      requestType = Type.ListKeys
+  )
+  public static OMResponse disallowListKeysResponseWithECReplicationConfig(
+      OMRequest req, OMResponse resp, ValidationContext ctx)
+      throws ServiceException {
+    if (!resp.hasListKeysResponse()) {
+      return resp;
+    }
+    List<KeyInfo> keys = resp.getListKeysResponse().getKeyInfoList();
+    for (KeyInfo key : keys) {
+      if (key.hasEcReplicationConfig()) {
+        resp = resp.toBuilder()
+            .setStatus(Status.NOT_SUPPORTED_OPERATION)
+            .setMessage("The list of keys contains keys with Erasure Coded"
+                + " replication set, hence the client is not able to"
+                + " represent all the keys returned. Please upgrade the"
+                + " client to get the list of keys.")
+            .clearListKeysResponse()
+            .build();
+      }
+    }
+    return resp;
+  }
+
   private ListTrashResponse listTrash(ListTrashRequest request,
       int clientVersion) throws IOException {
 
@@ -452,6 +515,36 @@
     return resp.build();
   }
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+      processingPhase = RequestProcessingPhase.POST_PROCESS,
+      requestType = Type.ListTrash
+  )
+  public static OMResponse disallowListTrashWithECReplicationConfig(
+      OMRequest req, OMResponse resp, ValidationContext ctx)
+      throws ServiceException {
+    if (!resp.hasListTrashResponse()) {
+      return resp;
+    }
+    List<RepeatedKeyInfo> repeatedKeys =
+        resp.getListTrashResponse().getDeletedKeysList();
+    for (RepeatedKeyInfo repeatedKey : repeatedKeys) {
+      for (KeyInfo key : repeatedKey.getKeyInfoList()) {
+        if (key.hasEcReplicationConfig()) {
+          resp = resp.toBuilder()
+              .setStatus(Status.NOT_SUPPORTED_OPERATION)
+              .setMessage("The list of keys contains keys with Erasure Coded"
+                  + " replication set, hence the client is not able to"
+                  + " represent all the keys returned. Please upgrade the"
+                  + " client to get the list of keys.")
+              .clearListTrashResponse()
+              .build();
+        }
+      }
+    }
+    return resp;
+  }
+
   private ServiceListResponse getServiceList(ServiceListRequest request)
       throws IOException {
     ServiceListResponse.Builder resp = ServiceListResponse.newBuilder();
@@ -500,13 +593,18 @@
 
     omPartInfoList.forEach(partInfo -> partInfoList.add(partInfo.getProto()));
 
-    response.setType(
-            omMultipartUploadListParts
-                    .getReplicationConfig()
-                    .getReplicationType());
-    response.setFactor(
-            ReplicationConfig.getLegacyFactor(
-                    omMultipartUploadListParts.getReplicationConfig()));
+    HddsProtos.ReplicationType repType = omMultipartUploadListParts
+        .getReplicationConfig()
+        .getReplicationType();
+    response.setType(repType);
+    if (repType == HddsProtos.ReplicationType.EC) {
+      response.setEcReplicationConfig(
+          ((ECReplicationConfig)omMultipartUploadListParts
+              .getReplicationConfig()).toProto());
+    } else {
+      response.setFactor(ReplicationConfig.getLegacyFactor(
+          omMultipartUploadListParts.getReplicationConfig()));
+    }
     response.setNextPartNumberMarker(
         omMultipartUploadListParts.getNextPartNumberMarker());
     response.setIsTruncated(omMultipartUploadListParts.isTruncated());
@@ -527,17 +625,27 @@
     List<MultipartUploadInfo> info = omMultipartUploadList
         .getUploads()
         .stream()
-        .map(upload -> MultipartUploadInfo.newBuilder()
-            .setVolumeName(upload.getVolumeName())
-            .setBucketName(upload.getBucketName())
-            .setKeyName(upload.getKeyName())
-            .setUploadId(upload.getUploadId())
-            .setType(upload.getReplicationConfig().getReplicationType())
-            .setFactor(
-                    ReplicationConfig.getLegacyFactor(
-                            upload.getReplicationConfig()))
-            .setCreationTime(upload.getCreationTime().toEpochMilli())
-            .build())
+        .map(upload -> {
+          MultipartUploadInfo.Builder bldr = MultipartUploadInfo.newBuilder()
+              .setVolumeName(upload.getVolumeName())
+              .setBucketName(upload.getBucketName())
+              .setKeyName(upload.getKeyName())
+              .setUploadId(upload.getUploadId());
+
+          HddsProtos.ReplicationType repType = upload.getReplicationConfig()
+              .getReplicationType();
+          bldr.setType(repType);
+          if (repType == HddsProtos.ReplicationType.EC) {
+            bldr.setEcReplicationConfig(
+                ((ECReplicationConfig)upload.getReplicationConfig())
+                    .toProto());
+          } else {
+            bldr.setFactor(ReplicationConfig.getLegacyFactor(
+                upload.getReplicationConfig()));
+          }
+          bldr.setCreationTime(upload.getCreationTime().toEpochMilli());
+          return bldr.build();
+        })
         .collect(Collectors.toList());
 
     ListMultipartUploadsResponse response =
@@ -564,6 +672,35 @@
     return rb.build();
   }
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+      processingPhase = RequestProcessingPhase.POST_PROCESS,
+      requestType = Type.GetFileStatus
+  )
+  public static OMResponse disallowGetFileStatusWithECReplicationConfig(
+      OMRequest req, OMResponse resp, ValidationContext ctx)
+      throws ServiceException {
+    if (!resp.hasGetFileStatusResponse()) {
+      return resp;
+    }
+    if (resp.getGetFileStatusResponse().getStatus().getKeyInfo()
+        .hasEcReplicationConfig()) {
+      resp = resp.toBuilder()
+          .setStatus(Status.NOT_SUPPORTED_OPERATION)
+          .setMessage("Key is a key with Erasure Coded replication, which"
+              + " the client can not understand."
+              + " Please upgrade the client before trying to read the key info"
+              + " for "
+              + req.getGetFileStatusRequest().getKeyArgs().getVolumeName()
+              + "/" + req.getGetFileStatusRequest().getKeyArgs().getBucketName()
+              + "/" + req.getGetFileStatusRequest().getKeyArgs().getKeyName()
+              + ".")
+          .clearGetFileStatusResponse()
+          .build();
+    }
+    return resp;
+  }
+
   private LookupFileResponse lookupFile(LookupFileRequest request,
       int clientVersion) throws IOException {
     KeyArgs keyArgs = request.getKeyArgs();
@@ -580,6 +717,34 @@
         .build();
   }
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+      processingPhase = RequestProcessingPhase.POST_PROCESS,
+      requestType = Type.LookupFile
+  )
+  public static OMResponse disallowLookupFileWithECReplicationConfig(
+      OMRequest req, OMResponse resp, ValidationContext ctx)
+      throws ServiceException {
+    if (!resp.hasLookupFileResponse()) {
+      return resp;
+    }
+    if (resp.getLookupFileResponse().getKeyInfo().hasEcReplicationConfig()) {
+      resp = resp.toBuilder()
+          .setStatus(Status.NOT_SUPPORTED_OPERATION)
+          .setMessage("Key is a key with Erasure Coded replication, which the"
+              + " client can not understand."
+              + " Please upgrade the client before trying to read the key info"
+              + " for "
+              + req.getLookupFileRequest().getKeyArgs().getVolumeName()
+              + "/" + req.getLookupFileRequest().getKeyArgs().getBucketName()
+              + "/" + req.getLookupFileRequest().getKeyArgs().getKeyName()
+              + ".")
+          .clearLookupFileResponse()
+          .build();
+    }
+    return resp;
+  }
+
   private ListStatusResponse listStatus(
       ListStatusRequest request, int clientVersion) throws IOException {
     KeyArgs keyArgs = request.getKeyArgs();
@@ -603,6 +768,34 @@
     return listStatusResponseBuilder.build();
   }
 
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+      processingPhase = RequestProcessingPhase.POST_PROCESS,
+      requestType = Type.ListStatus
+  )
+  public static OMResponse disallowListStatusResponseWithECReplicationConfig(
+      OMRequest req, OMResponse resp, ValidationContext ctx)
+      throws ServiceException {
+    if (!resp.hasListStatusResponse()) {
+      return resp;
+    }
+    List<OzoneFileStatusProto> statuses =
+        resp.getListStatusResponse().getStatusesList();
+    for (OzoneFileStatusProto status : statuses) {
+      if (status.getKeyInfo().hasEcReplicationConfig()) {
+        resp = resp.toBuilder()
+            .setStatus(Status.NOT_SUPPORTED_OPERATION)
+            .setMessage("The list of keys contains keys with Erasure Coded"
+                + " replication set, hence the client is not able to"
+                + " represent all the keys returned."
+                + " Please upgrade the client to get the list of keys.")
+            .clearListStatusResponse()
+            .build();
+      }
+    }
+    return resp;
+  }
+
   private FinalizeUpgradeProgressResponse reportUpgradeProgress(
       FinalizeUpgradeProgressRequest request) throws IOException {
     String upgradeClientId = request.getUpgradeClientId();
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneConfigUtil.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneConfigUtil.java
new file mode 100644
index 0000000..c888ee7
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneConfigUtil.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests the server side replication config preference logic.
+ */
+public class TestOzoneConfigUtil {
+  private ReplicationConfig ratis3ReplicationConfig =
+      RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+  private HddsProtos.ReplicationType noneType = HddsProtos.ReplicationType.NONE;
+  private HddsProtos.ReplicationFactor zeroFactor =
+      HddsProtos.ReplicationFactor.ZERO;
+  private HddsProtos.ECReplicationConfig clientECReplicationConfig =
+      new ECReplicationConfig("rs-3-2-1024K").toProto();
+  private DefaultReplicationConfig bucketECConfig =
+      new DefaultReplicationConfig(
+          new ECReplicationConfig(clientECReplicationConfig));
+
+  /**
+   * Tests EC bucket defaults.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWhenBucketHasEC() {
+    ReplicationConfig replicationConfig = OzoneConfigUtil
+        .resolveReplicationConfigPreference(noneType, zeroFactor,
+            clientECReplicationConfig, bucketECConfig, ratis3ReplicationConfig);
+    // Client has no preference, so we should bucket defaults as we passed.
+    Assert.assertEquals(bucketECConfig.getEcReplicationConfig(),
+        replicationConfig);
+  }
+
+  /**
+   * Tests server defaults.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWithNoClientAndBucketDefaults() {
+    ReplicationConfig replicationConfig = OzoneConfigUtil
+        .resolveReplicationConfigPreference(noneType, zeroFactor,
+            clientECReplicationConfig, null, ratis3ReplicationConfig);
+    // Client has no preference, no bucket defaults, so it should return server
+    // defaults.
+    Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * Tests client preference of EC.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWhenClientPassEC() {
+    ReplicationConfig replicationConfig = OzoneConfigUtil
+        .resolveReplicationConfigPreference(HddsProtos.ReplicationType.EC,
+            zeroFactor, clientECReplicationConfig, null,
+            ratis3ReplicationConfig);
+    // Client has preference of type EC, no bucket defaults, so it should return
+    // client preference.
+    Assert.assertEquals(new ECReplicationConfig("rs-3-2-1024K"),
+        replicationConfig);
+  }
+
+  /**
+   * Tests bucket ratis defaults.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWhenBucketHasEC3() {
+    DefaultReplicationConfig ratisBucketDefaults =
+        new DefaultReplicationConfig(ReplicationType.RATIS,
+            ReplicationFactor.THREE);
+    ReplicationConfig replicationConfig = OzoneConfigUtil
+        .resolveReplicationConfigPreference(noneType, zeroFactor,
+            clientECReplicationConfig, ratisBucketDefaults,
+            ratis3ReplicationConfig);
+    // Client has no preference of type and bucket has ratis defaults, so it
+    // should return ratis.
+    Assert.assertEquals(ratisBucketDefaults.getType().name(),
+        replicationConfig.getReplicationType().name());
+    Assert.assertEquals(ratisBucketDefaults.getFactor(),
+        ReplicationFactor.valueOf(replicationConfig.getRequiredNodes()));
+  }
+
+}
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java
index cb5bb23..f0af5e7 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java
@@ -24,6 +24,8 @@
 import java.util.List;
 import java.util.UUID;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
@@ -44,6 +46,7 @@
     .OMRequest;
 
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS;
 import static org.apache.hadoop.ozone.om.request.OMRequestTestUtils.addVolumeAndBucketToDB;
@@ -59,12 +62,33 @@
 
   @Test
   public void testPreExecuteWithNormalKey() throws Exception {
-    doPreExecute(createKeyRequest(false, 0));
+    ReplicationConfig ratis3Config =
+        ReplicationConfig.fromProtoTypeAndFactor(RATIS, THREE);
+    preExecuteTest(false, 0, ratis3Config);
+  }
+
+  @Test
+  public void testPreExecuteWithECKey() throws Exception {
+    ReplicationConfig ec3Plus2Config = new ECReplicationConfig("rs-3-2-1024k");
+    preExecuteTest(false, 0, ec3Plus2Config);
   }
 
   @Test
   public void testPreExecuteWithMultipartKey() throws Exception {
-    doPreExecute(createKeyRequest(true, 1));
+    ReplicationConfig ratis3Config =
+        ReplicationConfig.fromProtoTypeAndFactor(RATIS, THREE);
+    preExecuteTest(true, 1, ratis3Config);
+  }
+
+  private void preExecuteTest(boolean isMultipartKey, int partNumber,
+      ReplicationConfig repConfig) throws Exception {
+    long scmBlockSize = ozoneManager.getScmBlockSize();
+    for (int i = 0; i <= repConfig.getRequiredNodes(); i++) {
+      doPreExecute(createKeyRequest(isMultipartKey, partNumber,
+          scmBlockSize * i, repConfig));
+      doPreExecute(createKeyRequest(isMultipartKey, partNumber,
+          scmBlockSize * i + 1, repConfig));
+    }
   }
 
 
@@ -163,10 +187,13 @@
         omMetadataManager.getOpenKeyTable(bucketLayout).get(openKey);
 
     Assert.assertNotNull(omKeyInfo);
+    Assert.assertNotNull(omKeyInfo.getLatestVersionLocations());
 
+    // As our data size is 100, and scmBlockSize is default to 1000, so we
+    // shall have only one block.
     List<OmKeyLocationInfo> omKeyLocationInfoList =
         omKeyInfo.getLatestVersionLocations().getLocationList();
-    Assert.assertTrue(omKeyLocationInfoList.size() == 1);
+    Assert.assertEquals(1, omKeyLocationInfoList.size());
 
     OmKeyLocationInfo omKeyLocationInfo = omKeyLocationInfoList.get(0);
 
@@ -342,6 +369,12 @@
         modifiedOmRequest.getCreateKeyRequest();
 
     KeyArgs keyArgs = createKeyRequest.getKeyArgs();
+    int dataGroupSize = keyArgs.hasEcReplicationConfig() ?
+        keyArgs.getEcReplicationConfig().getData() : 1;
+    long blockSize = ozoneManager.getScmBlockSize();
+    long preAllocatedBlocks = Math.min(ozoneManager.getPreallocateBlocksMax(),
+        (keyArgs.getDataSize() - 1) / (blockSize * dataGroupSize) + 1);
+
     // Time should be set
     Assert.assertTrue(keyArgs.getModificationTime() > 0);
 
@@ -354,12 +387,10 @@
     if (!originalOMRequest.getCreateKeyRequest().getKeyArgs()
         .getIsMultipartKey()) {
 
-      // As our data size is 100, and scmBlockSize is default to 1000, so we
-      // shall have only one block.
-      List< OzoneManagerProtocolProtos.KeyLocation> keyLocations =
+      List<OzoneManagerProtocolProtos.KeyLocation> keyLocations =
           keyArgs.getKeyLocationsList();
       // KeyLocation should be set.
-      Assert.assertTrue(keyLocations.size() == 1);
+      Assert.assertEquals(preAllocatedBlocks, keyLocations.size());
       Assert.assertEquals(CONTAINER_ID,
           keyLocations.get(0).getBlockID().getContainerBlockID()
               .getContainerID());
@@ -373,7 +404,7 @@
       Assert.assertEquals(scmBlockSize, keyLocations.get(0).getLength());
     } else {
       // We don't create blocks for multipart key in createKey preExecute.
-      Assert.assertTrue(keyArgs.getKeyLocationsList().size() == 0);
+      Assert.assertEquals(0, keyArgs.getKeyLocationsList().size());
     }
 
     return modifiedOmRequest;
@@ -414,6 +445,36 @@
         .setCreateKeyRequest(createKeyRequest).build();
   }
 
+  private OMRequest createKeyRequest(boolean isMultipartKey, int partNumber,
+      long keyLength, ReplicationConfig repConfig) {
+
+    KeyArgs.Builder keyArgs = KeyArgs.newBuilder()
+        .setVolumeName(volumeName).setBucketName(bucketName)
+        .setKeyName(keyName).setIsMultipartKey(isMultipartKey)
+        .setType(repConfig.getReplicationType())
+        .setLatestVersionLocation(true)
+        .setDataSize(keyLength);
+
+    if (repConfig.getReplicationType() == EC) {
+      keyArgs.setEcReplicationConfig(
+          ((ECReplicationConfig) repConfig).toProto());
+    } else {
+      keyArgs.setFactor(ReplicationConfig.getLegacyFactor(repConfig));
+    }
+
+    if (isMultipartKey) {
+      keyArgs.setMultipartNumber(partNumber);
+    }
+
+    OzoneManagerProtocolProtos.CreateKeyRequest createKeyRequest =
+        CreateKeyRequest.newBuilder().setKeyArgs(keyArgs).build();
+
+    return OMRequest.newBuilder()
+        .setCmdType(OzoneManagerProtocolProtos.Type.CreateKey)
+        .setClientId(UUID.randomUUID().toString())
+        .setCreateKeyRequest(createKeyRequest).build();
+  }
+
   @Test
   public void testKeyCreateWithFileSystemPathsEnabled() throws Exception {
 
@@ -426,11 +487,11 @@
     addVolumeAndBucketToDB(volumeName, bucketName,
         omMetadataManager, getBucketLayout());
 
-    keyName = "dir1/dir2/dir3/file1";
+    String keyName = "dir1/dir2/dir3/file1";
     createAndCheck(keyName);
 
     // Key with leading '/'.
-    String keyName = "/a/b/c/file1";
+    keyName = "/a/b/c/file1";
     createAndCheck(keyName);
 
     // Commit openKey entry.
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
index 33e09ab..d538f20 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
@@ -24,11 +24,14 @@
 import java.util.UUID;
 
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.om.OzoneManagerPrepareState;
 import org.apache.hadoop.ozone.om.ResolvedBucket;
 import org.apache.hadoop.ozone.om.KeyManager;
 import org.apache.hadoop.ozone.om.KeyManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
@@ -134,6 +137,8 @@
     when(ozoneManager.isAdmin(any(String.class))).thenReturn(true);
     when(ozoneManager.isAdmin(any(UserGroupInformation.class)))
         .thenReturn(true);
+    when(ozoneManager.getBucketInfo(anyString(), anyString())).thenReturn(
+        new OmBucketInfo.Builder().setVolumeName("").setBucketName("").build());
     Mockito.doNothing().when(auditLogger).logWrite(any(AuditMessage.class));
 
     scmClient = Mockito.mock(ScmClient.class);
@@ -163,17 +168,21 @@
         .setNodes(new ArrayList<>())
         .build();
 
-    AllocatedBlock allocatedBlock =
-        new AllocatedBlock.Builder()
-            .setContainerBlockID(new ContainerBlockID(CONTAINER_ID, LOCAL_ID))
-            .setPipeline(pipeline).build();
+    AllocatedBlock.Builder blockBuilder = new AllocatedBlock.Builder()
+        .setPipeline(pipeline);
 
-    List<AllocatedBlock> allocatedBlocks = new ArrayList<>();
-
-    allocatedBlocks.add(allocatedBlock);
-
-    when(scmBlockLocationProtocol.allocateBlock(anyLong(), anyInt(), any(),
-        anyString(), any())).thenReturn(allocatedBlocks);
+    when(scmBlockLocationProtocol.allocateBlock(anyLong(), anyInt(),
+        any(ReplicationConfig.class),
+        anyString(), any(ExcludeList.class))).thenAnswer(invocation -> {
+          int num = invocation.getArgument(1);
+          List<AllocatedBlock> allocatedBlocks = new ArrayList<>(num);
+          for (int i = 0; i < num; i++) {
+            blockBuilder.setContainerBlockID(
+                new ContainerBlockID(CONTAINER_ID + i, LOCAL_ID + i));
+            allocatedBlocks.add(blockBuilder.build());
+          }
+          return allocatedBlocks;
+        });
 
 
     volumeName = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java
index 581bd2a..591c3bf 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java
@@ -22,6 +22,7 @@
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.junit.After;
@@ -83,6 +84,8 @@
     when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
     auditLogger = Mockito.mock(AuditLogger.class);
     when(ozoneManager.getAuditLogger()).thenReturn(auditLogger);
+    when(ozoneManager.getDefaultReplicationConfig()).thenReturn(
+        ReplicationConfig.getDefault(ozoneConfiguration));
     Mockito.doNothing().when(auditLogger).logWrite(any(AuditMessage.class));
     when(ozoneManager.resolveBucketLink(any(KeyArgs.class),
         any(OMClientRequest.class)))
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
index 4508eaf..8b97bba 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
@@ -16,7 +16,6 @@
  */
 package org.apache.hadoop.ozone.om.request.validation;
 
-import com.google.protobuf.ServiceException;
 import org.apache.hadoop.ozone.ClientVersion;
 import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting;
 import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.ValidationListener;
@@ -69,7 +68,7 @@
   }
 
   @Test(expected = NullPointerException.class)
-  public void testUsingRegistryWithoutLoading() throws ServiceException {
+  public void testUsingRegistryWithoutLoading() throws Exception {
     new RequestValidations()
         .fromPackage(PACKAGE)
         .withinContext(of(aFinalizedVersionManager()))
@@ -77,7 +76,7 @@
   }
 
   @Test(expected = NullPointerException.class)
-  public void testUsingRegistryWithoutContext() throws ServiceException {
+  public void testUsingRegistryWithoutContext() throws Exception {
     new RequestValidations()
         .fromPackage(PACKAGE)
         .load()
@@ -85,7 +84,7 @@
   }
 
   @Test
-  public void testUsingRegistryWithoutPackage() throws ServiceException {
+  public void testUsingRegistryWithoutPackage() throws Exception {
     new RequestValidations()
         .withinContext(of(aFinalizedVersionManager()))
         .load()
@@ -96,7 +95,7 @@
 
   @Test
   public void testNoPreValidationsWithoutValidationMethods()
-      throws ServiceException {
+      throws Exception {
     int omVersion = 0;
     ValidationContext ctx = of(aFinalizedVersionManager());
     RequestValidations validations = loadEmptyValidations(ctx);
@@ -108,7 +107,7 @@
 
   @Test
   public void testNoPostValidationsWithoutValidationMethods()
-      throws ServiceException {
+      throws Exception {
     ValidationContext ctx = of(aFinalizedVersionManager());
     RequestValidations validations = loadEmptyValidations(ctx);
 
@@ -120,7 +119,7 @@
 
   @Test
   public void testNoPreValidationsRunningForRequestTypeWithoutValidators()
-      throws ServiceException {
+      throws Exception {
     ValidationContext ctx = of(aFinalizedVersionManager());
     RequestValidations validations = loadValidations(ctx);
 
@@ -131,7 +130,7 @@
 
   @Test
   public void testNoPostValidationsAreRunningForRequestTypeWithoutValidators()
-      throws ServiceException {
+      throws Exception {
     ValidationContext ctx = of(aFinalizedVersionManager());
     RequestValidations validations = loadValidations(ctx);
 
@@ -142,14 +141,14 @@
   }
 
   @Test
-  public void testPreProcessorExceptionHandling() {
+  public void testPreProcessorExceptionHandling() throws Exception {
     ValidationContext ctx = of(aFinalizedVersionManager());
     RequestValidations validations = loadValidations(ctx);
 
     try {
       validations.validateRequest(aDeleteKeysRequest(olderClientVersion()));
       fail("ServiceException was expected but was not thrown.");
-    } catch (ServiceException ignored) { }
+    } catch (Exception ignored) { }
 
     validationListener.assertNumOfEvents(1);
     validationListener.assertExactListOfValidatorsCalled(
@@ -165,7 +164,7 @@
       validations.validateResponse(
           aDeleteKeysRequest(olderClientVersion()), aDeleteKeysResponse());
       fail("ServiceException was expected but was not thrown.");
-    } catch (ServiceException ignored) { }
+    } catch (Exception ignored) { }
 
     validationListener.assertNumOfEvents(1);
     validationListener.assertExactListOfValidatorsCalled(
@@ -174,7 +173,7 @@
 
   @Test
   public void testOldClientConditionIsRecognizedAndPreValidatorsApplied()
-      throws ServiceException {
+      throws Exception {
     ValidationContext ctx = of(aFinalizedVersionManager());
     RequestValidations validations = loadValidations(ctx);
 
@@ -187,7 +186,7 @@
 
   @Test
   public void testOldClientConditionIsRecognizedAndPostValidatorsApplied()
-      throws ServiceException {
+      throws Exception {
     ValidationContext ctx = of(aFinalizedVersionManager());
     RequestValidations validations = loadValidations(ctx);
 
@@ -202,7 +201,7 @@
 
   @Test
   public void testPreFinalizedWithOldClientConditionPreProcValidatorsApplied()
-      throws ServiceException {
+      throws Exception {
     ValidationContext ctx = of(anUnfinalizedVersionManager());
     RequestValidations validations = loadValidations(ctx);
 
@@ -216,7 +215,7 @@
 
   @Test
   public void testPreFinalizedWithOldClientConditionPostProcValidatorsApplied()
-      throws ServiceException {
+      throws Exception {
     ValidationContext ctx = of(anUnfinalizedVersionManager());
     RequestValidations validations = loadValidations(ctx);
 
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java
index f3b470c..f9d72b6 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java
@@ -21,6 +21,7 @@
 import com.google.common.collect.Iterators;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -135,6 +136,8 @@
         );
     when(om.getAclsEnabled()).thenReturn(false);
     when(om.getAuditLogger()).thenReturn(mock(AuditLogger.class));
+    when(om.getDefaultReplicationConfig()).thenReturn(ReplicationConfig
+        .getDefault(new OzoneConfiguration()));
     addVolumeToMetaTable(aVolumeArgs());
     addBucketToMetaTable(aBucketInfo());
   }
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
index 55bce59..bec00e9 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
@@ -21,11 +21,14 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
+import java.time.Clock;
+import java.time.ZoneOffset;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.HashSet;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.BlockLocation;
@@ -51,6 +54,7 @@
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.common.MonotonicClock;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
@@ -65,6 +69,7 @@
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -84,10 +89,15 @@
   private ObjectStore objectStore;
   private OzoneVolume volume;
   private OzoneBucket bucket;
-  private ReplicationConfig replicationConfig;
+  private ReplicationConfig bucketReplicationConfig;
+  // Client side configured replication config.
+  private ReplicationConfig clientConfiguredReplicationConfig;
   private boolean securityEnabled;
   private int configuredDnPort;
   private OzoneConfiguration config;
+  private long nextReplicationConfigRefreshTime;
+  private long bucketRepConfigRefreshPeriodMS;
+  private java.time.Clock clock = new MonotonicClock(ZoneOffset.UTC);
 
   /**
    * Create new OzoneClientAdapter implementation.
@@ -112,6 +122,11 @@
       throws IOException {
 
     OzoneConfiguration conf = OzoneConfiguration.of(hadoopConf);
+    bucketRepConfigRefreshPeriodMS = conf.getLong(
+        OzoneConfigKeys
+            .OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_MS,
+        OzoneConfigKeys
+            .OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_DEFAULT_MS);
     if (omHost == null && OmUtils.isServiceIdsDefined(conf)) {
       // When the host name or service id isn't given
       // but ozone.om.service.ids is defined, declare failure.
@@ -141,7 +156,8 @@
       this.securityEnabled = true;
     }
 
-    replicationConfig = ReplicationConfig.getDefault(conf);
+    clientConfiguredReplicationConfig =
+        OzoneClientUtils.getClientConfiguredReplicationConfig(conf);
 
     if (OmUtils.isOmHAServiceId(conf, omHost)) {
       // omHost is listed as one of the service ids in the config,
@@ -158,6 +174,9 @@
     objectStore = ozoneClient.getObjectStore();
     this.volume = objectStore.getVolume(volumeStr);
     this.bucket = volume.getBucket(bucketStr);
+    bucketReplicationConfig = this.bucket.getReplicationConfig();
+    nextReplicationConfigRefreshTime =
+        clock.millis() + bucketRepConfigRefreshPeriodMS;
 
     // resolve the bucket layout in case of Link Bucket
     BucketLayout resolvedBucketLayout =
@@ -172,9 +191,20 @@
     this.config = conf;
   }
 
+  /**
+   * This API returns the value what is configured at client side only. It could
+   * differ from the server side default values. If no replication config
+   * configured at client, it will return 3.
+   */
   @Override
   public short getDefaultReplication() {
-    return (short) replicationConfig.getRequiredNodes();
+    if (clientConfiguredReplicationConfig == null) {
+      // to provide backward compatibility, we are just retuning 3;
+      // However we need to handle with the correct behavior.
+      // TODO: Please see HDDS-5646
+      return (short) ReplicationFactor.THREE.getValue();
+    }
+    return (short) clientConfiguredReplicationConfig.getRequiredNodes();
   }
 
   @Override
@@ -207,21 +237,11 @@
       boolean overWrite, boolean recursive) throws IOException {
     incrementCounter(Statistic.OBJECTS_CREATED, 1);
     try {
-      OzoneOutputStream ozoneOutputStream = null;
-      if (replication == ReplicationFactor.ONE.getValue()
-          || replication == ReplicationFactor.THREE.getValue()) {
-
-        ReplicationConfig customReplicationConfig =
-            ReplicationConfig.adjustReplication(
-                replicationConfig, replication, config
-            );
-        ozoneOutputStream =
-            bucket.createFile(key, 0, customReplicationConfig, overWrite,
-                recursive);
-      } else {
-        ozoneOutputStream =
-            bucket.createFile(key, 0, replicationConfig, overWrite, recursive);
-      }
+      OzoneOutputStream ozoneOutputStream = bucket.createFile(key, 0,
+          OzoneClientUtils.resolveClientSideReplicationConfig(replication,
+              this.clientConfiguredReplicationConfig,
+              getReplicationConfigWithRefreshCheck(), config), overWrite,
+          recursive);
       return new OzoneFSOutputStream(ozoneOutputStream.getOutputStream());
     } catch (OMException ex) {
       if (ex.getResult() == OMException.ResultCodes.FILE_ALREADY_EXISTS
@@ -234,6 +254,17 @@
     }
   }
 
+  private ReplicationConfig getReplicationConfigWithRefreshCheck()
+      throws IOException {
+    if (clock.millis() > nextReplicationConfigRefreshTime) {
+      this.bucketReplicationConfig =
+          volume.getBucket(bucket.getName()).getReplicationConfig();
+      nextReplicationConfigRefreshTime =
+          clock.millis() + bucketRepConfigRefreshPeriodMS;
+    }
+    return this.bucketReplicationConfig;
+  }
+
   @Override
   public void renameKey(String key, String newKeyName) throws IOException {
     incrementCounter(Statistic.OBJECTS_RENAMED, 1);
@@ -399,6 +430,11 @@
     return objectStore.getCanonicalServiceName();
   }
 
+  @VisibleForTesting
+  void setClock(Clock monotonicClock) {
+    this.clock = monotonicClock;
+  }
+
   /**
    * Ozone Delegation Token Renewer.
    */
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java
index 35a027d..f880c5e 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java
@@ -101,7 +101,7 @@
   private OzoneClient ozoneClient;
   private ObjectStore objectStore;
   private ClientProtocol proxy;
-  private ReplicationConfig replicationConfig;
+  private ReplicationConfig clientConfiguredReplicationConfig;
   private boolean securityEnabled;
   private int configuredDnPort;
   private BucketLayout defaultOFSBucketLayout;
@@ -171,7 +171,8 @@
         this.securityEnabled = true;
       }
 
-      replicationConfig = ReplicationConfig.getDefault(conf);
+      clientConfiguredReplicationConfig =
+          OzoneClientUtils.getClientConfiguredReplicationConfig(conf);
 
       if (OmUtils.isOmHAServiceId(conf, omHost)) {
         // omHost is listed as one of the service ids in the config,
@@ -288,9 +289,20 @@
     return bucket;
   }
 
+  /**
+   * This API returns the value what is configured at client side only. It could
+   * differ from the server side default values. If no replication config
+   * configured at client, it will return 3.
+   */
   @Override
   public short getDefaultReplication() {
-    return (short) replicationConfig.getRequiredNodes();
+    if (clientConfiguredReplicationConfig == null) {
+      // to provide backward compatibility, we are just retuning 3;
+      // However we need to handle with the correct behavior.
+      // TODO: Please see HDDS-5646
+      return (short) ReplicationFactor.THREE.getValue();
+    }
+    return (short) clientConfiguredReplicationConfig.getRequiredNodes();
   }
 
   @Override
@@ -333,18 +345,10 @@
     try {
       // Hadoop CopyCommands class always sets recursive to true
       OzoneBucket bucket = getBucket(ofsPath, recursive);
-      OzoneOutputStream ozoneOutputStream = null;
-      if (replication == ReplicationFactor.ONE.getValue()
-          || replication == ReplicationFactor.THREE.getValue()) {
-
-        ozoneOutputStream = bucket.createFile(key, 0,
-            ReplicationConfig.adjustReplication(
-                replicationConfig, replication, config),
-            overWrite, recursive);
-      } else {
-        ozoneOutputStream =
-            bucket.createFile(key, 0, replicationConfig, overWrite, recursive);
-      }
+      OzoneOutputStream ozoneOutputStream = bucket.createFile(key, 0,
+          OzoneClientUtils.resolveClientSideReplicationConfig(replication,
+              this.clientConfiguredReplicationConfig,
+              bucket.getReplicationConfig(), config), overWrite, recursive);
       return new OzoneFSOutputStream(ozoneOutputStream.getOutputStream());
     } catch (OMException ex) {
       if (ex.getResult() == OMException.ResultCodes.FILE_ALREADY_EXISTS
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java
index c440a6b..14021da 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java
@@ -18,6 +18,11 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.ozone.client.checksum.BaseFileChecksumHelper;
@@ -32,6 +37,9 @@
 import java.io.IOException;
 import java.util.Set;
 
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.DETECTED_LOOP_IN_BUCKET_LINKS;
 
 /**
@@ -72,6 +80,122 @@
     return bucket.getBucketLayout();
   }
 
+  /**
+   * This API used to resolve the client side configuration preference for file
+   * system layer implementations.
+   *
+   * @param replication                - replication value passed from FS API.
+   * @param clientConfiguredReplConfig - Client side configured replication
+   *                                   config.
+   * @param bucketReplConfig           - server side bucket default replication
+   *                                  config.
+   * @param config                     - Ozone configuration object.
+   * @return client resolved replication config.
+   */
+  public static ReplicationConfig resolveClientSideReplicationConfig(
+      short replication, ReplicationConfig clientConfiguredReplConfig,
+      ReplicationConfig bucketReplConfig, ConfigurationSource config) {
+    ReplicationConfig clientDeterminedReplConfig = null;
+
+    boolean isECBucket = bucketReplConfig != null && bucketReplConfig
+        .getReplicationType() == HddsProtos.ReplicationType.EC;
+
+    // if bucket replication config configured with EC, we will give high
+    // preference to server side bucket defaults.
+    // Why we give high prefernce to EC is, there is no way for file system
+    // interfaces to pass EC replication. So, if one configures EC at bucket,
+    // we consider EC to take preference. in short, keys created from file
+    // system under EC bucket will always be EC'd.
+    if (isECBucket) {
+      // if bucket is EC, don't bother client provided configs, let's pass
+      // bucket config.
+      clientDeterminedReplConfig = bucketReplConfig;
+    } else {
+      // Let's validate the client side available replication configs.
+      boolean isReplicationInSupportedList =
+          (replication == ReplicationFactor.ONE
+              .getValue() || replication == ReplicationFactor.THREE.getValue());
+      if (isReplicationInSupportedList) {
+        if (clientConfiguredReplConfig != null) {
+          // Uses the replication(short value) passed from file system API and
+          // construct replication config object.
+          // In case if client explicitely configured EC in configurations, we
+          // always take EC as priority as EC replication can't be expressed in
+          // filesystem API.
+          clientDeterminedReplConfig = ReplicationConfig
+              .adjustReplication(clientConfiguredReplConfig, replication,
+                  config);
+        } else {
+          // In file system layers, replication parameter always passed.
+          // so, to respect the API provided replication value, we take RATIS as
+          // default type.
+          clientDeterminedReplConfig = ReplicationConfig
+              .parse(ReplicationType.RATIS, Short.toString(replication),
+                  config);
+        }
+      } else {
+        // API passed replication number is not in supported replication list.
+        // So, let's use whatever available in client side configured.
+        // By default it will be null, so server will use server defaults.
+        clientDeterminedReplConfig = clientConfiguredReplConfig;
+      }
+    }
+    return clientDeterminedReplConfig;
+  }
+
+  public static ReplicationConfig getClientConfiguredReplicationConfig(
+      ConfigurationSource config) {
+    String replication = config.get(OZONE_REPLICATION);
+    if (replication == null) {
+      return null;
+    }
+    return ReplicationConfig.parse(ReplicationType.valueOf(
+        config.get(OZONE_REPLICATION_TYPE, OZONE_REPLICATION_TYPE_DEFAULT)),
+        replication, config);
+  }
+
+  /**
+   * Gets the client side replication config by checking user passed values vs
+   * client configured values.
+   * @param userPassedType - User provided replication type.
+   * @param userPassedReplication - User provided replication.
+   * @param clientSideConfig - Client side configuration.
+   * @return ReplicationConfig.
+   */
+  public static ReplicationConfig validateAndGetClientReplicationConfig(
+      ReplicationType userPassedType, String userPassedReplication,
+      ConfigurationSource clientSideConfig) {
+    // Priority 1: User passed replication config values.
+    // Priority 2: Client side configured replication config values.
+    /* if above two are not available, we should just return null and clients
+     can pass null replication config to server. Now server will take the
+     decision of finding the replication config( either from bucket defaults
+     or server defaults). */
+    ReplicationType clientReplicationType = userPassedType;
+    String clientReplication = userPassedReplication;
+    String clientConfiguredDefaultType =
+        clientSideConfig.get(OZONE_REPLICATION_TYPE);
+    if (userPassedType == null && clientConfiguredDefaultType != null) {
+      clientReplicationType =
+          ReplicationType.valueOf(clientConfiguredDefaultType);
+    }
+
+    String clientConfiguredDefaultReplication =
+        clientSideConfig.get(OZONE_REPLICATION);
+    if (userPassedReplication == null
+        && clientConfiguredDefaultReplication != null) {
+      clientReplication = clientConfiguredDefaultReplication;
+    }
+
+    // if clientReplicationType or clientReplication is null, then we just pass
+    // replication config as null, so that server will take decision.
+    if (clientReplicationType == null || clientReplication == null) {
+      return null;
+    }
+    return ReplicationConfig
+        .parse(clientReplicationType, clientReplication, clientSideConfig);
+  }
+
   public static FileChecksum getFileChecksumWithCombineMode(OzoneVolume volume,
       OzoneBucket bucket, String keyName, long length,
       OzoneClientConfig.ChecksumCombineMode combineMode,
diff --git a/hadoop-ozone/ozonefs-common/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneClientUtils.java b/hadoop-ozone/ozonefs-common/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneClientUtils.java
index a9d4f98..561cc34 100644
--- a/hadoop-ozone/ozonefs-common/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneClientUtils.java
+++ b/hadoop-ozone/ozonefs-common/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneClientUtils.java
@@ -17,10 +17,18 @@
 package org.apache.hadoop.fs.ozone;
 
 import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -29,9 +37,16 @@
 import static org.mockito.Mockito.mock;
 
 /**
- * Unit tests for OzoneClientUtils.
+ * Tests the behavior of OzoneClientUtils APIs.
  */
 public class TestOzoneClientUtils {
+  private ReplicationConfig ecReplicationConfig =
+      new ECReplicationConfig("rs-3-2-1024K");
+  private ReplicationConfig ratis3ReplicationConfig =
+      RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+  private ReplicationConfig ratis1ReplicationConfig =
+      RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE);
+
   @Test(expected = IllegalArgumentException.class)
   public void testNegativeLength() throws IOException {
     OzoneVolume volume = mock(OzoneVolume.class);
@@ -56,4 +71,191 @@
 
     assertNull(checksum);
   }
-}
\ No newline at end of file
+
+  @Test
+  public void testResolveClientSideRepConfigWhenBucketHasEC() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .resolveClientSideReplicationConfig(
+            (short) 3, null,
+            ecReplicationConfig, new OzoneConfiguration());
+    // Bucket default is EC.
+    Assert.assertEquals(ecReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket replication is null and it should respect fs passed value.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWhenBucketHasNull() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .resolveClientSideReplicationConfig(
+            (short) 3, null, null,
+            new OzoneConfiguration());
+    // Passed replication is 3 - Ozone mapped replication is ratis THREE
+    Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket replication is null and it should return null if fs passed
+   * value is invalid.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWhenFSPassedReplicationIsInvalid() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .resolveClientSideReplicationConfig(
+            (short) -1, null, null,
+            new OzoneConfiguration());
+    // client configured value also null.
+    // This API caller should leave the decision to server.
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side values are not valid, we
+   * would just return null, so servers can make decision in this case.
+   */
+  @Test
+  public void testResolveRepConfWhenFSPassedIsInvalidButBucketDefaultNonEC() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .resolveClientSideReplicationConfig(
+            (short) -1, null, ratis3ReplicationConfig,
+            new OzoneConfiguration());
+    // Configured client config also null.
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side value is valid, we
+   * would should return client side valid value.
+   */
+  @Test
+  public void testResolveRepConfWhenFSPassedIsValidButBucketDefaultNonEC() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .resolveClientSideReplicationConfig(
+            (short) 1, null, ratis3ReplicationConfig,
+            new OzoneConfiguration());
+    // Passed value is replication one - Ozone mapped value is ratis ONE
+    Assert.assertEquals(ratis1ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket default is EC and client side value also valid, we would just
+   * return bucket default EC.
+   */
+  @Test
+  public void testResolveRepConfWhenFSPassedIsValidButBucketDefaultEC() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .resolveClientSideReplicationConfig(
+            (short) 3, ratis3ReplicationConfig,
+            ecReplicationConfig, new OzoneConfiguration());
+    // Bucket default is EC
+    Assert.assertEquals(ecReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side passed value also not valid
+   * but configured value is valid, we would just return configured value.
+   */
+  @Test
+  public void testResolveRepConfWhenFSPassedIsInvalidAndBucketDefaultNonEC() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .resolveClientSideReplicationConfig(
+            (short) -1, ratis3ReplicationConfig, ratis1ReplicationConfig,
+            new OzoneConfiguration());
+    // Configured value is ratis THREE
+    Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * Tests validateAndGetClientReplicationConfig with user passed valid config
+   * values.
+   */
+  @Test
+  public void testValidateAndGetRepConfWhenValidUserPassedValues() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .validateAndGetClientReplicationConfig(ReplicationType.RATIS, "1",
+            new OzoneConfiguration());
+    // Configured value is ratis ONE
+    Assert.assertEquals(ratis1ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * Tests validateAndGetClientReplicationConfig with user passed null values.
+   */
+  @Test
+  public void testValidateAndGetRepConfWhenValidUserPassedNullValues() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .validateAndGetClientReplicationConfig(null, null,
+            new OzoneConfiguration());
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * Tests validateAndGetClientReplicationConfig with user passed null values
+   * but client config has valid values.
+   */
+  @Test
+  public void testValidateAndGetRepConfWhenValidConfigValues() {
+    OzoneConfiguration clientSideConfig = new OzoneConfiguration();
+    clientSideConfig.set(OzoneConfigKeys.OZONE_REPLICATION_TYPE, "EC");
+    clientSideConfig.set(OzoneConfigKeys.OZONE_REPLICATION, "rs-3-2-1024K");
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .validateAndGetClientReplicationConfig(null, null, clientSideConfig);
+    Assert.assertEquals(ecReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * Tests validateAndGetClientReplicationConfig with user passed null values
+   * but client config has valid values.
+   */
+  @Test
+  public void testValidateAndGetRepConfWhenNullTypeFromUser() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .validateAndGetClientReplicationConfig(null, "3",
+            new OzoneConfiguration());
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * Tests validateAndGetClientReplicationConfig with user passed null
+   * replication but valid type.
+   */
+  @Test
+  public void testValidateAndGetRepConfWhenNullReplicationFromUser() {
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .validateAndGetClientReplicationConfig(ReplicationType.EC, null,
+            new OzoneConfiguration());
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * Tests validateAndGetClientReplicationConfig with user pass null values but
+   * config has only replication configured.
+   */
+  @Test
+  public void testValidateAndGetRepConfWhenNullTypeConfigValues() {
+    OzoneConfiguration clientSideConfig = new OzoneConfiguration();
+    clientSideConfig.set(OzoneConfigKeys.OZONE_REPLICATION, "rs-3-2-1024K");
+    //By default config values are null. Let's don't set type to keep it as
+    // null.
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .validateAndGetClientReplicationConfig(null, null, clientSideConfig);
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * Tests validateAndGetClientReplicationConfig with user pass null values but
+   * config has only type configured.
+   */
+  @Test
+  public void testValidateAndGetRepConfWhenNullReplicationConfigValues() {
+    OzoneConfiguration clientSideConfig = new OzoneConfiguration();
+    clientSideConfig.set(OzoneConfigKeys.OZONE_REPLICATION_TYPE, "EC");
+    //By default config values are null. Let's don't set replication to keep it
+    // as null.
+    ReplicationConfig replicationConfig = OzoneClientUtils
+        .validateAndGetClientReplicationConfig(null, null, clientSideConfig);
+    Assert.assertNull(replicationConfig);
+  }
+
+}
diff --git a/hadoop-ozone/pom.xml b/hadoop-ozone/pom.xml
index 04a967d..96c5bea 100644
--- a/hadoop-ozone/pom.xml
+++ b/hadoop-ozone/pom.xml
@@ -134,6 +134,11 @@
       </dependency>
       <dependency>
         <groupId>org.apache.ozone</groupId>
+        <artifactId>hdds-erasurecode</artifactId>
+        <version>${hdds.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.ozone</groupId>
         <artifactId>hdds-interface-client</artifactId>
         <version>${hdds.version}</version>
       </dependency>
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java
index 4e6ee3a..d4b126f 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.ozone.recon.api;
 
-import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
@@ -105,13 +104,9 @@
         try {
           Pipeline pipeline = pipelineManager.getPipeline(pipelineID);
           String leaderNode = pipeline.getLeaderNode().getHostName();
-          DatanodePipeline datanodePipeline = new DatanodePipeline(
-              pipelineID.getId(),
-              pipeline.getReplicationConfig().getReplicationType().toString(),
-              ReplicationConfig.getLegacyFactor(pipeline.getReplicationConfig())
-                  .getNumber(),
-              leaderNode
-          );
+          DatanodePipeline datanodePipeline =
+              new DatanodePipeline(pipelineID.getId(),
+                  pipeline.getReplicationConfig(), leaderNode);
           pipelines.add(datanodePipeline);
           if (datanode.getUuid().equals(pipeline.getLeaderId())) {
             leaderCount.getAndIncrement();
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PipelineEndpoint.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PipelineEndpoint.java
index d475be4..0595bbf 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PipelineEndpoint.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PipelineEndpoint.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.ozone.recon.api;
 
-import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager;
 import org.apache.hadoop.ozone.recon.MetricsServiceProviderFactory;
@@ -105,9 +104,7 @@
               .setDatanodes(datanodes)
               .setDuration(duration)
               .setStatus(pipeline.getPipelineState())
-              .setReplicationFactor(ReplicationConfig
-                  .getLegacyFactor(pipeline.getReplicationConfig()).getNumber())
-          .setReplicationType(pipeline.getType().toString());
+              .setReplicationConfig(pipeline.getReplicationConfig());
       // If any metrics service providers like Prometheus
       // is configured, then query it for metrics and populate
       // leader election count and last leader election time
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DatanodePipeline.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DatanodePipeline.java
index cc38cd5..e302cad 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DatanodePipeline.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DatanodePipeline.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.ozone.recon.api.types;
 
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+
 import java.util.UUID;
 
 /**
@@ -25,14 +27,14 @@
 public class DatanodePipeline {
   private UUID pipelineID;
   private String replicationType;
-  private int replicationFactor;
+  private String replicationFactor;
   private String leaderNode;
 
-  public DatanodePipeline(UUID pipelineID, String replicationType,
-                          int replicationFactor, String leaderNode) {
+  public DatanodePipeline(UUID pipelineID, ReplicationConfig replicationConfig,
+      String leaderNode) {
     this.pipelineID = pipelineID;
-    this.replicationType = replicationType;
-    this.replicationFactor = replicationFactor;
+    this.replicationType = replicationConfig.getReplicationType().toString();
+    this.replicationFactor = replicationConfig.getReplication();
     this.leaderNode = leaderNode;
   }
 
@@ -44,7 +46,7 @@
     return replicationType;
   }
 
-  public int getReplicationFactor() {
+  public String getReplicationFactor() {
     return replicationFactor;
   }
 
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/PipelineMetadata.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/PipelineMetadata.java
index 7af171f..e7facbf 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/PipelineMetadata.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/PipelineMetadata.java
@@ -19,6 +19,7 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState;
 
 import javax.xml.bind.annotation.XmlAccessType;
@@ -57,8 +58,10 @@
   @XmlElement(name = "replicationType")
   private String replicationType;
 
+  // TODO: name can be changed to just "replication". Currently EC replication
+  //  also showed with below parameter but in String format.
   @XmlElement(name = "replicationFactor")
-  private int replicationFactor;
+  private String replicationFactor;
 
   @XmlElement(name = "containers")
   private int containers;
@@ -95,7 +98,7 @@
     return replicationType;
   }
 
-  public int getReplicationFactor() {
+  public String getReplicationFactor() {
     return replicationFactor;
   }
 
@@ -139,7 +142,7 @@
     private long duration;
     private long leaderElections;
     private String replicationType;
-    private int replicationFactor;
+    private String replicationFactor;
     private int containers;
 
     public Builder() {
@@ -200,13 +203,9 @@
       return this;
     }
 
-    public Builder setReplicationType(String replicationType) {
-      this.replicationType = replicationType;
-      return this;
-    }
-
-    public Builder setReplicationFactor(int replicationFactor) {
-      this.replicationFactor = replicationFactor;
+    public Builder setReplicationConfig(ReplicationConfig replicationConfig) {
+      this.replicationType = replicationConfig.getReplicationType().toString();
+      this.replicationFactor = replicationConfig.getReplication();
       return this;
     }
 
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
index 02f2751..1f4fb53 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
@@ -19,10 +19,12 @@
 package org.apache.hadoop.ozone.recon.scm;
 
 import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.collections.map.DefaultedMap;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineFactory;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider;
@@ -52,10 +54,27 @@
 
     @Override
     public Pipeline create(ReplicationConfig config,
+        List<DatanodeDetails> excludedNodes,
+        List<DatanodeDetails> favoredNodes) {
+      // We don't expect this to be called at all. But adding this as a red
+      // flag for troubleshooting.
+      throw new UnsupportedOperationException(
+          "Trying to create pipeline in Recon, which is prohibited!");
+    }
+
+    @Override
+    public Pipeline create(ReplicationConfig config,
                            List<DatanodeDetails> nodes) {
       throw new UnsupportedOperationException(
           "Trying to create pipeline in Recon, which is prohibited!");
     }
+    
+    @Override
+    public Pipeline createForRead(ReplicationConfig config,
+        Set<ContainerReplica> replicas) {
+      throw new UnsupportedOperationException(
+          "Trying to create pipeline in Recon, which is prohibited!");
+    }
 
     @Override
     public void close(Pipeline pipeline) {
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx
index 62093d6..1781002 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx
@@ -82,7 +82,7 @@
 interface IPipeline {
   pipelineID: string;
   replicationType: string;
-  replicationFactor: number;
+  replicationFactor: string;
   leaderNode: string;
 }
 
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx
index 342a8bd..6810cba 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx
@@ -44,7 +44,7 @@
   lastLeaderElection: number;
   duration: number;
   leaderElections: number;
-  replicationFactor: number;
+  replicationFactor: string;
   containers: number;
 }
 
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java
index 68e6d38..fd01a11 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java
@@ -419,7 +419,7 @@
       Assert.assertEquals(1, datanodeMetadata.getPipelines().size());
       Assert.assertEquals(pipelineId,
           datanodeMetadata.getPipelines().get(0).getPipelineID().toString());
-      Assert.assertEquals(pipeline.getReplicationConfig().getRequiredNodes(),
+      Assert.assertEquals(pipeline.getReplicationConfig().getReplication(),
           datanodeMetadata.getPipelines().get(0).getReplicationFactor());
       Assert.assertEquals(pipeline.getType().toString(),
           datanodeMetadata.getPipelines().get(0).getReplicationType());
@@ -527,7 +527,7 @@
     Assert.assertEquals(1, pipelineMetadata.getDatanodes().size());
     Assert.assertEquals(pipeline.getType().toString(),
         pipelineMetadata.getReplicationType());
-    Assert.assertEquals(pipeline.getReplicationConfig().getRequiredNodes(),
+    Assert.assertEquals(pipeline.getReplicationConfig().getReplication(),
         pipelineMetadata.getReplicationFactor());
     Assert.assertEquals(datanodeDetails.getHostName(),
         pipelineMetadata.getLeaderNode());
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
index 808a8f0..d71a99b 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
@@ -309,8 +309,7 @@
               upload.getKeyName(),
               upload.getUploadId(),
               upload.getCreationTime(),
-              S3StorageType.fromReplicationType(upload.getReplicationType(),
-                  upload.getReplicationFactor())
+              S3StorageType.fromReplicationConfig(upload.getReplicationConfig())
           )));
       AUDIT.logReadSuccess(buildAuditMessageForSuccess(s3GAction, auditParams));
       getMetrics().incListMultipartUploadsSuccess();
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
index a852a3c..8824e98 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
@@ -53,7 +53,8 @@
 import java.util.Map;
 import java.util.OptionalLong;
 
-import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.StorageUnit;
@@ -78,6 +79,7 @@
 import org.apache.hadoop.ozone.s3.util.RangeHeader;
 import org.apache.hadoop.ozone.s3.util.RangeHeaderParserUtil;
 import org.apache.hadoop.ozone.s3.util.S3StorageType;
+import org.apache.hadoop.ozone.s3.util.S3Utils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.util.Time;
 
@@ -88,6 +90,9 @@
 
 import org.apache.commons.lang3.tuple.Pair;
 
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS;
 import static org.apache.hadoop.ozone.s3.S3GatewayConfigKeys.OZONE_S3G_CLIENT_BUFFER_SIZE_DEFAULT;
 import static org.apache.hadoop.ozone.s3.S3GatewayConfigKeys.OZONE_S3G_CLIENT_BUFFER_SIZE_KEY;
@@ -188,34 +193,24 @@
 
       copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER);
       storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
+      boolean storageTypeDefault = StringUtils.isEmpty(storageType);
 
-      S3StorageType s3StorageType;
-      boolean storageTypeDefault;
-      if (storageType == null || storageType.equals("")) {
-        s3StorageType = S3StorageType.getDefault(ozoneConfiguration);
-        storageTypeDefault = true;
-      } else {
-        s3StorageType = toS3StorageType(storageType);
-        storageTypeDefault = false;
-      }
-      ReplicationType replicationType = s3StorageType.getType();
-      ReplicationFactor replicationFactor = s3StorageType.getFactor();
+      // Normal put object
+      OzoneBucket bucket = getBucket(bucketName);
+      ReplicationConfig replicationConfig =
+          getReplicationConfig(bucket, storageType);
 
       if (copyHeader != null) {
         //Copy object, as copy source available.
         s3GAction = S3GAction.COPY_OBJECT;
         CopyObjectResponse copyObjectResponse = copyObject(
-            copyHeader, bucketName, keyPath, replicationType,
-            replicationFactor, storageTypeDefault);
+            copyHeader, bucket, keyPath, replicationConfig, storageTypeDefault);
         return Response.status(Status.OK).entity(copyObjectResponse).header(
             "Connection", "close").build();
       }
 
-      // Normal put object
-      OzoneBucket bucket = getBucket(bucketName);
-
-      output = bucket.createKey(keyPath, length, replicationType,
-          replicationFactor, new HashMap<>());
+      output =
+          bucket.createKey(keyPath, length, replicationConfig, new HashMap<>());
 
       if ("STREAMING-AWS4-HMAC-SHA256-PAYLOAD"
           .equals(headers.getHeaderString("x-amz-content-sha256"))) {
@@ -578,17 +573,11 @@
       OzoneBucket ozoneBucket = getBucket(bucket);
       String storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
 
-      S3StorageType s3StorageType;
-      if (storageType == null || storageType.equals("")) {
-        s3StorageType = S3StorageType.getDefault(ozoneConfiguration);
-      } else {
-        s3StorageType = toS3StorageType(storageType);
-      }
-      ReplicationType replicationType = s3StorageType.getType();
-      ReplicationFactor replicationFactor = s3StorageType.getFactor();
+      ReplicationConfig replicationConfig =
+          getReplicationConfig(ozoneBucket, storageType);
 
-      OmMultipartInfo multipartInfo = ozoneBucket
-          .initiateMultipartUpload(key, replicationType, replicationFactor);
+      OmMultipartInfo multipartInfo =
+          ozoneBucket.initiateMultipartUpload(key, replicationConfig);
 
       MultipartUploadInitiateResponse multipartUploadInitiateResponse = new
           MultipartUploadInitiateResponse();
@@ -619,6 +608,20 @@
     }
   }
 
+  private ReplicationConfig getReplicationConfig(OzoneBucket ozoneBucket,
+      String storageType) throws OS3Exception {
+    ReplicationConfig clientConfiguredReplicationConfig = null;
+    String replication = ozoneConfiguration.get(OZONE_REPLICATION);
+    if (replication != null) {
+      clientConfiguredReplicationConfig = ReplicationConfig.parse(
+          ReplicationType.valueOf(ozoneConfiguration
+              .get(OZONE_REPLICATION_TYPE, OZONE_REPLICATION_TYPE_DEFAULT)),
+          replication, ozoneConfiguration);
+    }
+    return S3Utils.resolveS3ClientSideReplicationConfig(storageType,
+        clientConfiguredReplicationConfig, ozoneBucket.getReplicationConfig());
+  }
+
   /**
    * Complete a multipart upload.
    */
@@ -818,9 +821,8 @@
       listPartsResponse.setPartNumberMarker(partNumberMarker);
       listPartsResponse.setTruncated(false);
 
-      listPartsResponse.setStorageClass(S3StorageType.fromReplicationType(
-          ozoneMultipartUploadPartListParts.getReplicationType(),
-          ozoneMultipartUploadPartListParts.getReplicationFactor()).toString());
+      listPartsResponse.setStorageClass(S3StorageType.fromReplicationConfig(
+          ozoneMultipartUploadPartListParts.getReplicationConfig()).toString());
 
       if (ozoneMultipartUploadPartListParts.isTruncated()) {
         listPartsResponse.setTruncated(
@@ -858,10 +860,9 @@
   }
 
   private CopyObjectResponse copyObject(String copyHeader,
-                                        String destBucket,
+                                        OzoneBucket destBucket,
                                         String destkey,
-                                        ReplicationType replicationType,
-                                        ReplicationFactor replicationFactor,
+                                        ReplicationConfig replicationConfig,
                                         boolean storageTypeDefault)
       throws OS3Exception, IOException {
 
@@ -875,7 +876,8 @@
     try {
       // Checking whether we trying to copying to it self.
 
-      if (sourceBucket.equals(destBucket) && sourceKey.equals(destkey)) {
+      if (sourceBucket.equals(destBucket.getName()) && sourceKey
+          .equals(destkey)) {
         // When copying to same storage type when storage type is provided,
         // we should not throw exception, as aws cli checks if any of the
         // options like storage type are provided or not when source and
@@ -901,15 +903,15 @@
 
 
       OzoneBucket sourceOzoneBucket = getBucket(sourceBucket);
-      OzoneBucket destOzoneBucket = getBucket(destBucket);
+      OzoneBucket destOzoneBucket = destBucket;
 
       OzoneKeyDetails sourceKeyDetails = sourceOzoneBucket.getKey(sourceKey);
       long sourceKeyLen = sourceKeyDetails.getDataSize();
 
       sourceInputStream = sourceOzoneBucket.readKey(sourceKey);
 
-      destOutputStream = destOzoneBucket.createKey(destkey, sourceKeyLen,
-          replicationType, replicationFactor, new HashMap<>());
+      destOutputStream = destOzoneBucket
+          .createKey(destkey, sourceKeyLen, replicationConfig, new HashMap<>());
 
       IOUtils.copy(sourceInputStream, destOutputStream);
 
@@ -977,15 +979,6 @@
     }
   }
 
-  private static S3StorageType toS3StorageType(String storageType)
-      throws OS3Exception {
-    try {
-      return S3StorageType.valueOf(storageType);
-    } catch (IllegalArgumentException ex) {
-      throw newError(INVALID_ARGUMENT, storageType, ex);
-    }
-  }
-
   private static int parsePartNumberMarker(String partNumberMarker) {
     int partMarker = 0;
     if (partNumberMarker != null) {
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3StorageType.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3StorageType.java
index 21b3b8d..ae42e81 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3StorageType.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3StorageType.java
@@ -18,9 +18,12 @@
 
 package org.apache.hadoop.ozone.s3.util;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 
 /**
@@ -70,10 +73,12 @@
         ? REDUCED_REDUNDANCY : STANDARD;
   }
 
-  public static S3StorageType fromReplicationType(
-      ReplicationType replicationType, ReplicationFactor factor) {
-    if ((replicationType == ReplicationType.STAND_ALONE) ||
-        (factor == ReplicationFactor.ONE)) {
+  public static S3StorageType fromReplicationConfig(ReplicationConfig config) {
+    if (config instanceof ECReplicationConfig) {
+      return S3StorageType.STANDARD;
+    }
+    if (config.getReplicationType() == HddsProtos.ReplicationType.STAND_ALONE ||
+        config.getRequiredNodes() == 1) {
       return S3StorageType.REDUCED_REDUNDANCY;
     } else {
       return S3StorageType.STANDARD;
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
index 7a07a89..5f110e8 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.ozone.s3.util;
 
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.commons.lang3.StringUtils;
 
 import java.io.UnsupportedEncodingException;
@@ -26,6 +31,8 @@
 import java.util.TreeMap;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.INVALID_ARGUMENT;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.newError;
 
 /**
  * Utilities.
@@ -61,4 +68,71 @@
   private S3Utils() {
     // no instances
   }
+
+  /**
+   * This API used to resolve the client side configuration preference for file
+   * system layer implementations.
+   *
+   * @param s3StorageTypeHeader        - s3 user passed storage type
+   *                                   header.
+   * @param clientConfiguredReplConfig - Client side configured replication
+   *                                   config.
+   * @param bucketReplConfig           - server side bucket default replication
+   *                                   config.
+   * @return client resolved replication config.
+   */
+  public static ReplicationConfig resolveS3ClientSideReplicationConfig(
+      String s3StorageTypeHeader, ReplicationConfig clientConfiguredReplConfig,
+      ReplicationConfig bucketReplConfig)
+      throws OS3Exception {
+    ReplicationConfig clientDeterminedReplConfig = null;
+
+    // Let's map the user provided s3 storage type header to ozone s3 storage
+    // type.
+    S3StorageType s3StorageType = null;
+    if (s3StorageTypeHeader != null && !s3StorageTypeHeader.equals("")) {
+      s3StorageType = toS3StorageType(s3StorageTypeHeader);
+    }
+
+    boolean isECBucket = bucketReplConfig != null && bucketReplConfig
+        .getReplicationType() == HddsProtos.ReplicationType.EC;
+
+    // if bucket replication config configured with EC, we will give high
+    // preference to server side bucket defaults.
+    // Why we give high preference to EC is, there is no way for file system
+    // interfaces to pass EC replication. So, if one configures EC at bucket,
+    // we consider EC to take preference. in short, keys created from file
+    // system under EC bucket will always be EC'd.
+    if (isECBucket) {
+      // if bucket is EC, don't bother client provided configs, let's pass
+      // bucket config.
+      clientDeterminedReplConfig = bucketReplConfig;
+    } else {
+      // Let's validate the client side available replication configs.
+      boolean isUserPassedReplicationInSupportedList =
+          s3StorageType != null && (s3StorageType.getFactor()
+              .getValue() == ReplicationFactor.ONE.getValue() || s3StorageType
+              .getFactor().getValue() == ReplicationFactor.THREE.getValue());
+      if (isUserPassedReplicationInSupportedList) {
+        clientDeterminedReplConfig = ReplicationConfig.fromProtoTypeAndFactor(
+            ReplicationType.toProto(s3StorageType.getType()),
+            ReplicationFactor.toProto(s3StorageType.getFactor()));
+      } else {
+        // API passed replication number is not in supported replication list.
+        // So, let's use whatever available in client side configured.
+        // By default it will be null, so server will use server defaults.
+        clientDeterminedReplConfig = clientConfiguredReplConfig;
+      }
+    }
+    return clientDeterminedReplConfig;
+  }
+
+  public static S3StorageType toS3StorageType(String storageType)
+      throws OS3Exception {
+    try {
+      return S3StorageType.valueOf(storageType);
+    } catch (IllegalArgumentException ex) {
+      throw newError(INVALID_ARGUMENT, storageType, ex);
+    }
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
index ad12e65..20e506d 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
@@ -32,10 +32,10 @@
 import java.util.stream.Collectors;
 
 import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
-import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.OzoneAcl;
@@ -62,6 +62,7 @@
   private Map<String, Map<Integer, Part>> partList = new HashMap<>();
 
   private ArrayList<OzoneAcl> aclList = new ArrayList<>();
+  private ReplicationConfig replicationConfig;
 
   /**
    * Constructs OzoneBucket instance.
@@ -79,11 +80,11 @@
       long creationTime) {
     super(volumeName,
         bucketName,
-        StandaloneReplicationConfig
-            .getInstance(HddsProtos.ReplicationFactor.ONE),
+        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE),
         storageType,
         versioning,
         creationTime);
+    this.replicationConfig = super.getReplicationConfig();
   }
 
   @Override
@@ -120,6 +121,37 @@
   }
 
   @Override
+  public OzoneOutputStream createKey(String key, long size,
+      ReplicationConfig rConfig, Map<String, String> metadata)
+      throws IOException {
+    final ReplicationConfig repConfig;
+    if (rConfig == null) {
+      repConfig = getReplicationConfig();
+    } else {
+      repConfig = rConfig;
+    }
+    ReplicationConfig finalReplicationCon = repConfig;
+    ByteArrayOutputStream byteArrayOutputStream =
+        new ByteArrayOutputStream((int) size) {
+          @Override
+          public void close() throws IOException {
+            keyContents.put(key, toByteArray());
+            keyDetails.put(key, new OzoneKeyDetails(
+                getVolumeName(),
+                getName(),
+                key,
+                size,
+                System.currentTimeMillis(),
+                System.currentTimeMillis(),
+                new ArrayList<>(), finalReplicationCon, metadata, null
+            ));
+            super.close();
+          }
+        };
+    return new OzoneOutputStream(byteArrayOutputStream);
+  }
+
+  @Override
   public OzoneInputStream readKey(String key) throws IOException {
     return new OzoneInputStream(new ByteArrayInputStream(keyContents.get(key)));
   }
@@ -143,10 +175,7 @@
           ozoneKeyDetails.getDataSize(),
           ozoneKeyDetails.getCreationTime().toEpochMilli(),
           ozoneKeyDetails.getModificationTime().toEpochMilli(),
-          ReplicationConfig.fromTypeAndFactor(
-              ozoneKeyDetails.getReplicationType(),
-              ReplicationFactor.valueOf(ozoneKeyDetails.getReplicationFactor())
-          ));
+          ozoneKeyDetails.getReplicationConfig());
     } else {
       throw new OMException(ResultCodes.KEY_NOT_FOUND);
     }
@@ -196,6 +225,14 @@
   }
 
   @Override
+  public OmMultipartInfo initiateMultipartUpload(String keyName,
+      ReplicationConfig repConfig) throws IOException {
+    String uploadID = UUID.randomUUID().toString();
+    multipartUploadIdMap.put(keyName, uploadID);
+    return new OmMultipartInfo(getVolumeName(), getName(), keyName, uploadID);
+  }
+
+  @Override
   public OzoneOutputStream createMultipartKey(String key, long size,
                                               int partNumber, String uploadID)
       throws IOException {
@@ -278,8 +315,9 @@
     List<PartInfo> partInfoList = new ArrayList<>();
 
     if (partList.get(key) == null) {
-      return new OzoneMultipartUploadPartListParts(ReplicationType.RATIS,
-          ReplicationFactor.ONE, 0, false);
+      return new OzoneMultipartUploadPartListParts(
+          RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+          0, false);
     } else {
       Map<Integer, Part> partMap = partList.get(key);
       Iterator<Map.Entry<Integer, Part>> partIterator =
@@ -308,8 +346,7 @@
       }
 
       OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
-          new OzoneMultipartUploadPartListParts(ReplicationType.RATIS,
-              ReplicationFactor.ONE,
+          new OzoneMultipartUploadPartListParts(replicationConfig,
               nextPartNumberMarker, truncated);
       ozoneMultipartUploadPartListParts.addAllParts(partInfoList);
 
@@ -359,4 +396,14 @@
       return content.clone();
     }
   }
+
+  @Override
+  public void setReplicationConfig(ReplicationConfig replicationConfig) {
+    this.replicationConfig = replicationConfig;
+  }
+
+  @Override
+  public ReplicationConfig getReplicationConfig() {
+    return this.replicationConfig;
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestInitiateMultipartUpload.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestInitiateMultipartUpload.java
index 2fa396f..8bd0b84 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestInitiateMultipartUpload.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestInitiateMultipartUpload.java
@@ -20,10 +20,12 @@
 
 package org.apache.hadoop.ozone.s3.endpoint;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientStub;
+import org.jetbrains.annotations.NotNull;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -53,10 +55,7 @@
     when(headers.getHeaderString(STORAGE_CLASS_HEADER)).thenReturn(
         "STANDARD");
 
-    ObjectEndpoint rest = new ObjectEndpoint();
-    rest.setHeaders(headers);
-    rest.setClient(client);
-    rest.setOzoneConfiguration(new OzoneConfiguration());
+    ObjectEndpoint rest = getObjectEndpoint(client, headers);
 
     Response response = rest.initializeMultipartUpload(bucket, key);
 
@@ -74,4 +73,32 @@
     assertNotNull(multipartUploadInitiateResponse.getUploadID());
     assertNotEquals(multipartUploadInitiateResponse.getUploadID(), uploadID);
   }
+
+  @Test
+  public void testInitiateMultipartUploadWithECKey() throws Exception {
+    String bucket = OzoneConsts.S3_BUCKET;
+    String key = OzoneConsts.KEY;
+    OzoneClient client = new OzoneClientStub();
+    client.getObjectStore().createS3Bucket(bucket);
+    HttpHeaders headers = Mockito.mock(HttpHeaders.class);
+    ObjectEndpoint rest = getObjectEndpoint(client, headers);
+    client.getObjectStore().getS3Bucket(bucket)
+        .setReplicationConfig(new ECReplicationConfig("rs-3-2-1024K"));
+    Response response = rest.initializeMultipartUpload(bucket, key);
+
+    assertEquals(200, response.getStatus());
+    MultipartUploadInitiateResponse multipartUploadInitiateResponse =
+        (MultipartUploadInitiateResponse) response.getEntity();
+    assertNotNull(multipartUploadInitiateResponse.getUploadID());
+  }
+
+  @NotNull
+  private ObjectEndpoint getObjectEndpoint(OzoneClient client,
+      HttpHeaders headers) {
+    ObjectEndpoint rest = new ObjectEndpoint();
+    rest.setHeaders(headers);
+    rest.setClient(client);
+    rest.setOzoneConfiguration(new OzoneConfiguration());
+    return rest;
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java
index cf44854..170981ff 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java
@@ -25,6 +25,7 @@
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.client.OzoneClient;
@@ -101,6 +102,34 @@
   }
 
   @Test
+  public void testPutObjectWithECReplicationConfig()
+      throws IOException, OS3Exception {
+    //GIVEN
+    HttpHeaders headers = Mockito.mock(HttpHeaders.class);
+    ByteArrayInputStream body =
+        new ByteArrayInputStream(CONTENT.getBytes(UTF_8));
+    objectEndpoint.setHeaders(headers);
+    ECReplicationConfig ecReplicationConfig =
+        new ECReplicationConfig("rs-3-2-1024K");
+    clientStub.getObjectStore().getS3Bucket(bucketName)
+        .setReplicationConfig(ecReplicationConfig);
+    Response response = objectEndpoint.put(bucketName, keyName, CONTENT
+        .length(), 1, null, body);
+
+    Assert.assertEquals(ecReplicationConfig,
+        clientStub.getObjectStore().getS3Bucket(bucketName).getKey(keyName)
+            .getReplicationConfig());
+    OzoneInputStream ozoneInputStream =
+        clientStub.getObjectStore().getS3Bucket(bucketName)
+            .readKey(keyName);
+    String keyContent =
+        IOUtils.toString(ozoneInputStream, UTF_8);
+
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(CONTENT, keyContent);
+  }
+
+  @Test
   public void testPutObjectWithSignedChunks() throws IOException, OS3Exception {
     //GIVEN
     HttpHeaders headers = Mockito.mock(HttpHeaders.class);
@@ -251,11 +280,11 @@
 
     objectEndpoint.put(bucketName, keyName, CONTENT
             .length(), 1, null, body);
-
     OzoneKeyDetails key =
         clientStub.getObjectStore().getS3Bucket(bucketName)
             .getKey(keyName);
 
+
     //default type is set
     Assert.assertEquals(ReplicationType.RATIS, key.getReplicationType());
   }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
index ca20dd8..07ba62c 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
@@ -268,7 +268,7 @@
   public void testPutKey() throws IOException {
     Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket);
     doThrow(exception).when(bucket)
-        .createKey(anyString(), anyLong(), any(), any(), any());
+        .createKey(anyString(), anyLong(), any(), any());
     ObjectEndpoint objectEndpoint = new ObjectEndpoint();
     objectEndpoint.setClient(client);
     objectEndpoint.setHeaders(headers);
@@ -304,8 +304,7 @@
   @Test
   public void testMultiUploadKey() throws IOException {
     Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket);
-    doThrow(exception).when(bucket)
-        .initiateMultipartUpload(anyString(), any(), any());
+    doThrow(exception).when(bucket).initiateMultipartUpload(anyString(), any());
     ObjectEndpoint objectEndpoint = new ObjectEndpoint();
     objectEndpoint.setClient(client);
     objectEndpoint.setHeaders(headers);
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3Utils.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3Utils.java
new file mode 100644
index 0000000..6b8c9fc
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3Utils.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.s3.util;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests the S3Utils APIs.
+ */
+public class TestS3Utils {
+  private ReplicationConfig ecReplicationConfig =
+      new ECReplicationConfig("rs-3-2-1024K");
+  private ReplicationConfig ratis3ReplicationConfig =
+      RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+  private ReplicationConfig ratis1ReplicationConfig =
+      RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE);
+
+  @Test
+  public void testResolveClientSideRepConfigWhenBucketHasEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(S3StorageType.STANDARD.name(),
+            null, ecReplicationConfig);
+    // Bucket default is EC.
+    Assert.assertEquals(ecReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket replication is null and it should respect user passed value.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWhenBucketHasNull()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(S3StorageType.STANDARD.name(),
+            null, null);
+    // Passed replication is 3 - Ozone mapped replication is ratis THREE
+    Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket replication is null and it should return null if user passed
+   * value is invalid.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWhenUserPassedReplicationIsEmpty()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig =
+        S3Utils.resolveS3ClientSideReplicationConfig("", null, null);
+    // client configured value also null.
+    // This API caller should leave the decision to server.
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side values are not valid, we
+   * would just return null, so servers can make decision in this case.
+   */
+  @Test
+  public void testResolveRepConfWhenUserPassedIsInvalidButBucketDefaultNonEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(null, null,
+            ratis3ReplicationConfig);
+    // Configured client config also null.
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side value is valid, we
+   * would should return client side valid value.
+   */
+  @Test
+  public void testResolveRepConfWhenUserPassedIsValidButBucketDefaultNonEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(
+            S3StorageType.REDUCED_REDUNDANCY.name(), null,
+            ratis3ReplicationConfig);
+    // Passed value is replication one - Ozone mapped value is ratis ONE
+    Assert.assertEquals(ratis1ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket default is EC and client side value also valid, we would just
+   * return bucket default EC.
+   */
+  @Test
+  public void testResolveRepConfWhenUserPassedIsValidButBucketDefaultEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(S3StorageType.STANDARD.name(),
+            ratis3ReplicationConfig, ecReplicationConfig);
+    // Bucket default is EC
+    Assert.assertEquals(ecReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side passed value also not valid
+   * but configured value is valid, we would just return configured value.
+   */
+  @Test
+  public void testResolveRepConfWhenUserPassedIsInvalidAndBucketDefaultNonEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(null, ratis3ReplicationConfig,
+            ratis1ReplicationConfig);
+    // Configured value is ratis THREE
+    Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side passed value also not valid
+   * but configured value is valid, we would just return configured value.
+   */
+  @Test(expected = OS3Exception.class)
+  public void testResolveRepConfWhenUserPassedIsInvalid() throws OS3Exception {
+    S3Utils.resolveS3ClientSideReplicationConfig("INVALID",
+        ratis3ReplicationConfig, ratis1ReplicationConfig);
+  }
+
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
index a7b330c..e983229 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
@@ -27,6 +27,7 @@
 import java.util.function.Consumer;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import java.util.stream.Stream;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -345,19 +346,24 @@
     return HAUtils.getScmContainerClient(ozoneConf);
   }
 
+  @SuppressWarnings("java:S3864") // Stream.peek (for debug)
   public static Pipeline findPipelineForTest(String pipelineId,
       StorageContainerLocationProtocol client, Logger log) throws IOException {
-    List<Pipeline> pipelines = client.listPipelines();
+    Stream<Pipeline> pipelines = client.listPipelines().stream();
     Pipeline pipeline;
+    if (log.isDebugEnabled()) {
+      pipelines = pipelines
+          .peek(p -> log.debug("Found pipeline {}", p.getId().getId()));
+    }
     if (pipelineId != null && pipelineId.length() > 0) {
-      pipeline = pipelines.stream()
+      pipeline = pipelines
           .filter(p -> p.getId().toString().equals(pipelineId))
           .findFirst()
           .orElseThrow(() -> new IllegalArgumentException(
               "Pipeline ID is defined, but there is no such pipeline: "
                   + pipelineId));
     } else {
-      pipeline = pipelines.stream()
+      pipeline = pipelines
           .filter(p -> p.getReplicationConfig().getRequiredNodes() == 3)
           .findFirst()
           .orElseThrow(() -> new IllegalArgumentException(
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java
index 5d86bc5..b290da2 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java
@@ -18,15 +18,12 @@
 
 import java.io.IOException;
 import java.util.concurrent.Callable;
-import java.util.stream.Stream;
 
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
 import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -79,7 +76,6 @@
 
 
   @Override
-  @SuppressWarnings("java:S3864") // Stream.peek (for debug)
   public Void call() throws Exception {
 
     init();
@@ -91,33 +87,9 @@
       );
     }
 
-    try (StorageContainerLocationProtocol scmLocationClient =
+    try (StorageContainerLocationProtocol scmClient =
                  createStorageContainerLocationClient(ozoneConf)) {
-      Stream<Pipeline> pipelines = scmLocationClient.listPipelines().stream();
-      if (LOG.isDebugEnabled()) {
-        pipelines = pipelines
-            .peek(p -> LOG.debug("Found pipeline {}", p.getId()));
-      }
-      Pipeline pipeline;
-      if (pipelineId != null && pipelineId.length() > 0) {
-        pipeline = pipelines
-              .filter(p -> p.getId().getId().toString().equals(pipelineId))
-              .findFirst()
-              .orElseThrow(() -> new IllegalArgumentException(
-                      "Pipeline ID is defined, but there is no such pipeline: "
-                              + pipelineId));
-
-      } else {
-        pipeline = pipelines
-            .filter(
-                p -> ReplicationConfig.getLegacyFactor(p.getReplicationConfig())
-                    == HddsProtos.ReplicationFactor.THREE)
-              .findFirst()
-              .orElseThrow(() -> new IllegalArgumentException(
-                      "Pipeline ID is NOT defined, and no pipeline " +
-                              "has been found with factor=THREE"));
-        LOG.info("Using pipeline {}", pipeline.getId());
-      }
+      Pipeline pipeline = findPipelineForTest(pipelineId, scmClient, LOG);
 
       try (XceiverClientManager xceiverClientManager =
                    new XceiverClientManager(ozoneConf)) {
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FreonReplicationOptions.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FreonReplicationOptions.java
new file mode 100644
index 0000000..3e5ac58
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FreonReplicationOptions.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.freon;
+
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.ozone.shell.ReplicationOptions;
+import picocli.CommandLine.Model.CommandSpec;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Spec;
+
+import java.util.Optional;
+
+import static picocli.CommandLine.Spec.Target.MIXEE;
+
+/**
+ * Options for specifying replication config for Freon.
+ */
+public class FreonReplicationOptions extends ReplicationOptions {
+
+  private static final String FACTOR_OPT = "--factor";
+
+  private ReplicationFactor factor;
+
+  @Spec(MIXEE)
+  private CommandSpec spec;
+
+  @Option(names = { "-F", FACTOR_OPT },
+      description = "[deprecated] Replication factor (ONE, THREE)",
+      defaultValue = "THREE"
+  )
+  public void setFactor(ReplicationFactor factor) {
+    this.factor = factor;
+  }
+
+  // -t is already taken for number of threads
+  @Option(names = {"--type", "--replication-type"},
+      description = "Replication type. Supported types are: RATIS, EC")
+  @Override
+  public void setType(ReplicationType type) {
+    super.setType(type);
+  }
+
+  /**
+   * Support legacy --factor option.
+   */
+  @Override
+  public Optional<ReplicationConfig> fromParams(ConfigurationSource conf) {
+    if (spec.commandLine().getParseResult().hasMatchedOption(FACTOR_OPT)) {
+      return Optional.of(ReplicationConfig.fromTypeAndFactor(
+          ReplicationType.RATIS, factor));
+    }
+
+    return super.fromParams(conf);
+  }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java
index 53a7705..28d3328 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java
@@ -20,9 +20,8 @@
 import java.util.concurrent.Callable;
 
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs.Builder;
@@ -33,6 +32,7 @@
 import com.codahale.metrics.Timer;
 import org.apache.hadoop.security.UserGroupInformation;
 import picocli.CommandLine.Command;
+import picocli.CommandLine.Mixin;
 import picocli.CommandLine.Option;
 
 import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.ALL;
@@ -61,11 +61,8 @@
       defaultValue = "bucket1")
   private String bucketName;
 
-  @Option(names = { "-F", "--factor" },
-      description = "Replication factor (ONE, THREE)",
-      defaultValue = "THREE"
-  )
-  private ReplicationFactor factor = ReplicationFactor.THREE;
+  @Mixin
+  private FreonReplicationOptions replication;
 
   @Option(
       names = "--om-service-id",
@@ -76,19 +73,20 @@
   private OzoneManagerProtocol ozoneManagerClient;
 
   private Timer timer;
+  private ReplicationConfig replicationConfig;
 
   @Override
   public Void call() throws Exception {
     init();
 
-    OzoneConfiguration ozoneConfiguration = createOzoneConfiguration();
+    OzoneConfiguration conf = createOzoneConfiguration();
+    replicationConfig = replication.fromParams(conf).orElse(null);
 
-    try (OzoneClient rpcClient = createOzoneClient(omServiceID,
-        ozoneConfiguration)) {
+    try (OzoneClient rpcClient = createOzoneClient(omServiceID, conf)) {
 
       ensureVolumeAndBucketExist(rpcClient, volumeName, bucketName);
 
-      ozoneManagerClient = createOmClient(ozoneConfiguration, omServiceID);
+      ozoneManagerClient = createOmClient(conf, omServiceID);
 
       timer = getMetrics().timer("key-create");
 
@@ -107,8 +105,8 @@
     OmKeyArgs keyArgs = new Builder()
         .setBucketName(bucketName)
         .setVolumeName(volumeName)
-        .setReplicationConfig(RatisReplicationConfig.getInstance(factor))
         .setKeyName(generateObjectName(counter))
+        .setReplicationConfig(replicationConfig)
         .setLocationInfoList(new ArrayList<>())
         .setAcls(OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroupNames(),
             ALL, ALL))
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyGenerator.java
index cf5e260..74cd0d0 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyGenerator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyGenerator.java
@@ -22,14 +22,14 @@
 import java.util.concurrent.Callable;
 
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneClient;
 
 import com.codahale.metrics.Timer;
 import picocli.CommandLine.Command;
+import picocli.CommandLine.Mixin;
 import picocli.CommandLine.Option;
 
 /**
@@ -66,23 +66,20 @@
       defaultValue = "4096")
   private int bufferSize;
 
-  @Option(names = { "-F", "--factor" },
-      description = "Replication factor (ONE, THREE)",
-      defaultValue = "THREE"
-  )
-  private ReplicationFactor factor = ReplicationFactor.THREE;
-
-  @Option(
-      names = "--om-service-id",
+  @Option(names = "--om-service-id",
       description = "OM Service ID"
   )
-  private String omServiceID = null;
+  private String omServiceID;
+
+  @Mixin
+  private FreonReplicationOptions replication;
 
   private Timer timer;
 
   private OzoneBucket bucket;
   private ContentGenerator contentGenerator;
   private Map<String, String> metadata;
+  private ReplicationConfig replicationConfig;
 
   @Override
   public Void call() throws Exception {
@@ -94,6 +91,8 @@
     contentGenerator = new ContentGenerator(keySize, bufferSize);
     metadata = new HashMap<>();
 
+    replicationConfig = replication.fromParamsOrConfig(ozoneConfiguration);
+
     try (OzoneClient rpcClient = createOzoneClient(omServiceID,
         ozoneConfiguration)) {
       ensureVolumeAndBucketExist(rpcClient, volumeName, bucketName);
@@ -112,7 +111,7 @@
 
     timer.time(() -> {
       try (OutputStream stream = bucket.createKey(key, keySize,
-              ReplicationType.RATIS, factor, metadata)) {
+          replicationConfig, metadata)) {
         contentGenerator.write(stream);
         stream.flush();
       }
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyValidator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyValidator.java
index 03de15c..1784ec3 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyValidator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyValidator.java
@@ -93,6 +93,8 @@
 
     runTests(this::validateKey);
 
+    rpcClient.close();
+
     return null;
   }
 
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
index 18ab210..75b1743 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
@@ -41,8 +41,7 @@
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.StringUtils;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -72,6 +71,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import picocli.CommandLine.Command;
+import picocli.CommandLine.Mixin;
 import picocli.CommandLine.Option;
 import picocli.CommandLine.ParentCommand;
 
@@ -177,20 +177,8 @@
   )
   private String jsonDir;
 
-  @Option(
-      names = {"--replication-type", "--replicationType"},
-      description = "Replication type (STAND_ALONE, RATIS). Full name " +
-          "--replicationType will be removed in later versions.",
-      defaultValue = "STAND_ALONE"
-  )
-  private ReplicationType type = ReplicationType.STAND_ALONE;
-
-  @Option(
-      names = "--factor",
-      description = "Replication factor (ONE, THREE)",
-      defaultValue = "ONE"
-  )
-  private ReplicationFactor factor = ReplicationFactor.ONE;
+  @Mixin
+  private FreonReplicationOptions replication;
 
   @Option(
       names = "--om-service-id",
@@ -205,6 +193,8 @@
   )
   private boolean cleanObjects = false;
 
+  private ReplicationConfig replicationConfig;
+
   private int threadPoolSize;
 
   private OzoneClient ozoneClient;
@@ -248,6 +238,7 @@
   private ProgressBar progressbar;
 
   RandomKeyGenerator() {
+    // for picocli
   }
 
   @VisibleForTesting
@@ -290,18 +281,19 @@
 
   @Override
   public Void call() throws Exception {
-    if (ozoneConfiguration != null) {
-      if (!ozoneConfiguration.getBoolean(
-          HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA,
-          HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA_DEFAULT)) {
-        LOG.info("Override validateWrites to false, because "
-            + HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA + " is set to false.");
-        validateWrites = false;
-      }
-      init(ozoneConfiguration);
-    } else {
-      init(freon.createOzoneConfiguration());
+    if (ozoneConfiguration == null) {
+      ozoneConfiguration = freon.createOzoneConfiguration();
     }
+    if (!ozoneConfiguration.getBoolean(
+        HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA,
+        HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA_DEFAULT)) {
+      LOG.info("Override validateWrites to false, because "
+          + HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA + " is set to false.");
+      validateWrites = false;
+    }
+    init(ozoneConfiguration);
+
+    replicationConfig = replication.fromParamsOrConfig(ozoneConfiguration);
 
     keyValueBuffer = StringUtils.string2Bytes(
         RandomStringUtils.randomAscii(bufferSize));
@@ -491,8 +483,8 @@
     out.println("Number of Volumes created: " + numberOfVolumesCreated);
     out.println("Number of Buckets created: " + numberOfBucketsCreated);
     out.println("Number of Keys added: " + numberOfKeysAdded);
-    out.println("Ratis replication factor: " + factor.name());
-    out.println("Ratis replication type: " + type.name());
+    out.println("Replication: " + replicationConfig.getReplication());
+    out.println("Replication type: " + replicationConfig.getReplicationType());
     out.println(
         "Average Time spent in volume creation: " + prettyAverageVolumeTime);
     out.println(
@@ -812,8 +804,8 @@
     try {
       try (AutoCloseable scope = TracingUtil.createActivatedSpan("createKey")) {
         long keyCreateStart = System.nanoTime();
-        try (OzoneOutputStream os = bucket.createKey(keyName, keySize, type,
-            factor, new HashMap<>())) {
+        try (OzoneOutputStream os = bucket.createKey(keyName, keySize,
+            replicationConfig, new HashMap<>())) {
           long keyCreationDuration = System.nanoTime() - keyCreateStart;
           histograms.get(FreonOps.KEY_CREATE.ordinal())
               .update(keyCreationDuration);
@@ -936,7 +928,7 @@
     private int numOfThreads;
     private String dataWritten;
     private String execTime;
-    private String replicationFactor;
+    private String replication;
     private String replicationType;
 
     private long keySize;
@@ -969,8 +961,8 @@
       this.keySize = RandomKeyGenerator.this.keySize;
       this.bufferSize = RandomKeyGenerator.this.bufferSize;
       this.jobStartTime = Time.formatTime(RandomKeyGenerator.this.jobStartTime);
-      this.replicationFactor = RandomKeyGenerator.this.factor.name();
-      this.replicationType = RandomKeyGenerator.this.type.name();
+      replicationType = replicationConfig.getReplicationType().name();
+      replication = replicationConfig.getReplication();
 
       long totalBytes =
           (long) numOfVolumes * numOfBuckets * numOfKeys * keySize;
@@ -1107,8 +1099,8 @@
       return execTime;
     }
 
-    public String getReplicationFactor() {
-      return replicationFactor;
+    public String getReplication() {
+      return replication;
     }
 
     public String getReplicationType() {
@@ -1229,52 +1221,7 @@
   }
 
   @VisibleForTesting
-  public void setNumOfVolumes(int numOfVolumes) {
-    this.numOfVolumes = numOfVolumes;
-  }
-
-  @VisibleForTesting
-  public void setNumOfBuckets(int numOfBuckets) {
-    this.numOfBuckets = numOfBuckets;
-  }
-
-  @VisibleForTesting
-  public void setNumOfKeys(int numOfKeys) {
-    this.numOfKeys = numOfKeys;
-  }
-
-  @VisibleForTesting
-  public void setNumOfThreads(int numOfThreads) {
-    this.numOfThreads = numOfThreads;
-  }
-
-  @VisibleForTesting
-  public void setKeySize(long keySize) {
-    this.keySize = keySize;
-  }
-
-  @VisibleForTesting
-  public void setType(ReplicationType type) {
-    this.type = type;
-  }
-
-  @VisibleForTesting
-  public void setFactor(ReplicationFactor factor) {
-    this.factor = factor;
-  }
-
-  @VisibleForTesting
-  public void setValidateWrites(boolean validateWrites) {
-    this.validateWrites = validateWrites;
-  }
-
-  @VisibleForTesting
   public int getThreadPoolSize() {
     return threadPoolSize;
   }
-
-  @VisibleForTesting
-  public void setCleanObjects(boolean cleanObjects) {
-    this.cleanObjects = cleanObjects;
-  }
 }
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ReplicationOptions.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ReplicationOptions.java
new file mode 100644
index 0000000..bb4e0a7
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ReplicationOptions.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.shell;
+
+import org.apache.hadoop.fs.ozone.OzoneClientUtils;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import picocli.CommandLine;
+
+import java.util.Optional;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE;
+
+/**
+ * Common options for specifying replication config: specialized for
+ * Ozone Shell and Freon commands.
+ */
+public abstract class ReplicationOptions {
+
+  private ReplicationType type;
+  private String replication;
+
+  public Optional<ReplicationConfig> fromParams(ConfigurationSource conf) {
+    if (replication == null && type == null) {
+      return Optional.empty();
+    }
+
+    // special case for backward compatibility: factor from config for RATIS
+    if (ReplicationType.RATIS == type && replication == null) {
+      replication = conf.get(OZONE_REPLICATION, OZONE_REPLICATION_DEFAULT);
+    }
+
+    return Optional.of(
+        ReplicationConfig.parseWithoutFallback(type, replication, conf));
+  }
+
+  public Optional<ReplicationConfig> fromConfig(ConfigurationSource conf) {
+    ReplicationType defaultType = Optional
+        .ofNullable(conf.get(OZONE_REPLICATION_TYPE))
+        .map(ReplicationType::valueOf)
+        .orElse(null);
+    String defaultReplication =
+        conf.get(OZONE_REPLICATION, OZONE_REPLICATION_DEFAULT);
+
+    return Optional.ofNullable(
+        OzoneClientUtils.validateAndGetClientReplicationConfig(
+            defaultType, defaultReplication, conf));
+  }
+
+  public ReplicationConfig fromParamsOrConfig(ConfigurationSource conf) {
+    return fromParams(conf)
+        .orElseGet(() -> fromConfig(conf)
+            .orElse(null));
+  }
+
+  @CommandLine.Option(names = {"--replication", "-r"},
+      description = "Replication definition. Valid values are replication"
+          + " type-specific.  For RATIS: ONE or THREE."
+          + " In case of EC, pass CODEC-DATA-PARITY-CHUNKSIZE, "
+          + " e.g. rs-3-2-1024k, rs-6-3-1024k, rs-10-4-1024k")
+  public void setReplication(String replication) {
+    this.replication = replication;
+  }
+
+  // Option is defined in subclasses
+  public void setType(ReplicationType type) {
+    this.type = type;
+  }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ShellReplicationOptions.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ShellReplicationOptions.java
new file mode 100644
index 0000000..3ee20c8
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ShellReplicationOptions.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.shell;
+
+import org.apache.hadoop.hdds.client.ReplicationType;
+import picocli.CommandLine.Option;
+
+/**
+ * Options for specifying replication config in 'ozone shell' commands.
+ */
+public class ShellReplicationOptions extends ReplicationOptions {
+
+  @Option(names = {"-t", "--type", "--replication-type"},
+      description = "Replication type. Supported types are: RATIS, EC")
+  @Override
+  public void setType(ReplicationType type) {
+    super.setType(type);
+  }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/BucketCommands.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/BucketCommands.java
index 2de2290..454660e 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/BucketCommands.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/BucketCommands.java
@@ -49,6 +49,7 @@
         GetAclBucketHandler.class,
         SetAclBucketHandler.class,
         ClearQuotaHandler.class,
+        SetReplicationConfigHandler.class,
         UpdateBucketHandler.class
     },
     mixinStandardHelpOptions = true,
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
index e1592e5..ea66429 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.shell.bucket;
 
 import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -29,6 +30,7 @@
 import org.apache.hadoop.ozone.shell.OzoneAddress;
 
 import org.apache.hadoop.ozone.shell.SetSpaceQuotaOptions;
+import org.apache.hadoop.ozone.shell.ShellReplicationOptions;
 import org.apache.hadoop.security.UserGroupInformation;
 import picocli.CommandLine;
 import picocli.CommandLine.Command;
@@ -65,6 +67,9 @@
   private AllowedBucketLayouts allowedBucketLayout;
 
   @CommandLine.Mixin
+  private ShellReplicationOptions replication;
+
+  @CommandLine.Mixin
   private SetSpaceQuotaOptions quotaOptions;
 
   /**
@@ -103,6 +108,9 @@
       }
     }
 
+    replication.fromParams(getConf()).ifPresent(config ->
+        bb.setDefaultReplicationConfig(new DefaultReplicationConfig(config)));
+
     if (!Strings.isNullOrEmpty(quotaOptions.getQuotaInBytes())) {
       bb.setQuotaInBytes(OzoneQuota.parseSpaceQuota(
           quotaOptions.getQuotaInBytes()).getQuotaInBytes());
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/SetReplicationConfigHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/SetReplicationConfigHandler.java
new file mode 100644
index 0000000..45d66fd
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/SetReplicationConfigHandler.java
@@ -0,0 +1,54 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.ozone.OzoneIllegalArgumentException;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.ShellReplicationOptions;
+import picocli.CommandLine;
+
+import java.io.IOException;
+
+/**
+ * set replication configuration of the bucket.
+ */
+@CommandLine.Command(name = "set-replication-config",
+    description = "Set replication config on bucket")
+public class SetReplicationConfigHandler extends BucketHandler {
+
+  @CommandLine.Mixin
+  private ShellReplicationOptions replication;
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+    ReplicationConfig replicationConfig = replication.fromParams(getConf())
+        .orElseThrow(() -> new OzoneIllegalArgumentException(
+            "Replication type and config must be specified."));
+
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    OzoneBucket bucket =
+        client.getObjectStore().getVolume(volumeName).getBucket(bucketName);
+    bucket.setReplicationConfig(replicationConfig);
+  }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/CopyKeyHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/CopyKeyHandler.java
index 5c0aaa3..81d2dbc 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/CopyKeyHandler.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/CopyKeyHandler.java
@@ -25,7 +25,6 @@
 
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
-import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -34,12 +33,14 @@
 import org.apache.hadoop.ozone.client.OzoneKeyDetails;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.ShellReplicationOptions;
 import org.apache.hadoop.ozone.shell.bucket.BucketHandler;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY;
+
 import picocli.CommandLine.Command;
-import picocli.CommandLine.Option;
+import picocli.CommandLine.Mixin;
 import picocli.CommandLine.Parameters;
 
 /**
@@ -58,17 +59,8 @@
       description = "The new desired name of the key")
   private String toKey;
 
-  @Option(names = {"-r", "--replication"},
-      description =
-          "Replication configuration of the new key. (this is replication "
-              + "specific. for RATIS/STANDALONE you can use ONE or THREE) "
-              + "Default is specified in the cluster-wide config.")
-  private String replication;
-
-  @Option(names = {"-t", "--type"},
-      description = "Replication type of the new key. (use RATIS or " +
-          "STAND_ALONE) Default is specified in the cluster-wide config.")
-  private ReplicationType replicationType;
+  @Mixin
+  private ShellReplicationOptions replication;
 
   @Override
   protected void execute(OzoneClient client, OzoneAddress address)
@@ -81,7 +73,7 @@
     OzoneBucket bucket = vol.getBucket(bucketName);
 
     ReplicationConfig replicationConfig =
-        ReplicationConfig.parse(replicationType, replication, getConf());
+        replication.fromParamsOrConfig(getConf());
 
     OzoneKeyDetails keyDetail = bucket.getKey(fromKey);
     Map<String, String> keyMetadata = new HashMap<>(keyDetail.getMetadata());
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java
index b7fbdd1..7d7885d 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java
@@ -28,7 +28,6 @@
 
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
-import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -40,8 +39,10 @@
 import org.apache.commons.codec.digest.DigestUtils;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY;
+
+import org.apache.hadoop.ozone.shell.ShellReplicationOptions;
 import picocli.CommandLine.Command;
-import picocli.CommandLine.Option;
+import picocli.CommandLine.Mixin;
 import picocli.CommandLine.Parameters;
 
 /**
@@ -54,17 +55,8 @@
   @Parameters(index = "1", arity = "1..1", description = "File to upload")
   private String fileName;
 
-  @Option(names = {"-r", "--replication"},
-      description =
-          "Replication configuration of the new key. (this is replication "
-              + "specific. for RATIS/STANDALONE you can use ONE or THREE) "
-              + "Default is specified in the cluster-wide config.")
-  private String replication;
-
-  @Option(names = {"-t", "--type"},
-      description = "Replication type of the new key. (use RATIS or " +
-          "STAND_ALONE) Default is specified in the cluster-wide config.")
-  private ReplicationType replicationType;
+  @Mixin
+  private ShellReplicationOptions replication;
 
   @Override
   protected void execute(OzoneClient client, OzoneAddress address)
@@ -84,7 +76,7 @@
     }
 
     ReplicationConfig replicationConfig =
-        ReplicationConfig.parse(replicationType, replication, getConf());
+        replication.fromParamsOrConfig(getConf());
 
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     OzoneBucket bucket = vol.getBucket(bucketName);
diff --git a/pom.xml b/pom.xml
index b31df37..fd0578c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1175,6 +1175,12 @@
         <scope>test</scope>
       </dependency>
       <dependency>
+        <groupId>org.junit.jupiter</groupId>
+        <artifactId>junit-jupiter-params</artifactId>
+        <version>${junit.jupiter.version}</version>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
         <groupId>org.junit.vintage</groupId>
         <artifactId>junit-vintage-engine</artifactId>
         <version>${junit.jupiter.version}</version>