This is an automated email from the ASF dual-hosted git repository.

umamahesh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 1944b2cda9 HDDS-6624. Make ECBlockReconstructedStripeInputStream 
available for datanodes (#3353)
1944b2cda9 is described below

commit 1944b2cda92ce44aa6e2c94b2fdf7a6aba8cebff
Author: Doroszlai, Attila <[email protected]>
AuthorDate: Wed Apr 27 17:41:54 2022 +0200

    HDDS-6624. Make ECBlockReconstructedStripeInputStream available for 
datanodes (#3353)
---
 hadoop-hdds/client/pom.xml                         |   4 +
 .../ozone/client/io/BadDataLocationException.java  |   0
 .../ozone/client/io/BlockInputStreamFactory.java   |   4 +-
 .../client/io/BlockInputStreamFactoryImpl.java     |   4 +-
 .../hadoop/ozone/client/io/ECBlockInputStream.java |  12 +-
 .../ozone/client/io/ECBlockInputStreamFactory.java |   4 +-
 .../client/io/ECBlockInputStreamFactoryImpl.java   |   4 +-
 .../ozone/client/io/ECBlockInputStreamProxy.java   |   6 +-
 .../client/io/ECBlockReconstructedInputStream.java |   0
 .../io/ECBlockReconstructedStripeInputStream.java  |   4 +-
 .../client/io/InsufficientLocationsException.java  |   0
 .../hadoop/ozone/client/io/package-info.java       |  24 +--
 .../hadoop/hdds/scm/storage/BlockLocationInfo.java | 113 +++--------
 hadoop-hdds/pom.xml                                |   6 +
 .../hadoop/ozone/om/helpers/OmKeyLocationInfo.java | 212 +++++----------------
 .../ozone/client/rpc/read/ECStreamTestUtil.java    |  10 +-
 .../client/rpc/read/TestECBlockInputStream.java    |  36 ++--
 .../rpc/read/TestECBlockInputStreamProxy.java      |  24 +--
 .../read/TestECBlockReconstructedInputStream.java  |   4 +-
 .../TestECBlockReconstructedStripeInputStream.java |  26 +--
 20 files changed, 150 insertions(+), 347 deletions(-)

diff --git a/hadoop-hdds/client/pom.xml b/hadoop-hdds/client/pom.xml
index 073bd9dc31..9f2116c96f 100644
--- a/hadoop-hdds/client/pom.xml
+++ b/hadoop-hdds/client/pom.xml
@@ -45,6 +45,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd";>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-erasurecode</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-core</artifactId>
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.java
 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.java
similarity index 100%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.java
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
similarity index 95%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
index d1bf7f306c..6703216016 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
+++ 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
@@ -22,8 +22,8 @@ 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.BlockLocationInfo;
 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;
@@ -47,7 +47,7 @@ public interface BlockInputStreamFactory {
    * @return BlockExtendedInputStream of the correct type.
    */
   BlockExtendedInputStream create(ReplicationConfig repConfig,
-      OmKeyLocationInfo blockInfo, Pipeline pipeline,
+      BlockLocationInfo 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-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
similarity index 97%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
index 104e5bcaff..ba05ec2ed8 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
+++ 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
@@ -25,10 +25,10 @@ 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.BlockLocationInfo;
 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;
@@ -75,7 +75,7 @@ public class BlockInputStreamFactoryImpl implements 
BlockInputStreamFactory {
    * @return BlockExtendedInputStream of the correct type.
    */
   public BlockExtendedInputStream create(ReplicationConfig repConfig,
-      OmKeyLocationInfo blockInfo, Pipeline pipeline,
+      BlockLocationInfo 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/ECBlockInputStream.java
 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
similarity index 97%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
index b0e97553e3..70bd3847c2 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
+++ 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
@@ -28,8 +28,8 @@ 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.BlockLocationInfo;
 import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,7 +53,7 @@ public class ECBlockInputStream extends 
BlockExtendedInputStream {
   private final boolean verifyChecksum;
   private final XceiverClientFactory xceiverClientFactory;
   private final Function<BlockID, Pipeline> refreshFunction;
-  private final OmKeyLocationInfo blockInfo;
+  private final BlockLocationInfo blockInfo;
   private final DatanodeDetails[] dataLocations;
   private final BlockExtendedInputStream[] blockStreams;
   private final int maxLocations;
@@ -62,10 +62,6 @@ public class ECBlockInputStream extends 
BlockExtendedInputStream {
   private boolean closed = false;
   private boolean seeked = false;
 
-  protected OmKeyLocationInfo getBlockInfo() {
-    return blockInfo;
-  }
-
   protected ECReplicationConfig getRepConfig() {
     return repConfig;
   }
@@ -109,7 +105,7 @@ public class ECBlockInputStream extends 
BlockExtendedInputStream {
   }
 
   public ECBlockInputStream(ECReplicationConfig repConfig,
-      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      BlockLocationInfo blockInfo, boolean verifyChecksum,
       XceiverClientFactory xceiverClientFactory, Function<BlockID,
       Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
     this.repConfig = repConfig;
@@ -174,7 +170,7 @@ public class ECBlockInputStream extends 
BlockExtendedInputStream {
           .setState(Pipeline.PipelineState.CLOSED)
           .build();
 
-      OmKeyLocationInfo blkInfo = new OmKeyLocationInfo.Builder()
+      BlockLocationInfo blkInfo = new BlockLocationInfo.Builder()
           .setBlockID(blockInfo.getBlockID())
           .setLength(internalBlockLength(locationIndex + 1))
           .setPipeline(blockInfo.getPipeline())
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
similarity index 95%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
index 6c39e935d4..c9d2b76a78 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
+++ 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
@@ -23,7 +23,7 @@ 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 org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
 
 import java.util.List;
 import java.util.function.Function;
@@ -52,7 +52,7 @@ public interface ECBlockInputStreamFactory {
    */
   BlockExtendedInputStream create(boolean missingLocations,
       List<DatanodeDetails> failedLocations, ReplicationConfig repConfig,
-      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      BlockLocationInfo 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-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java
similarity index 97%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java
index 470df0c1af..efc3b31c84 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java
+++ 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java
@@ -24,8 +24,8 @@ 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.BlockLocationInfo;
 import org.apache.hadoop.io.ByteBufferPool;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 
 import java.util.List;
 import java.util.concurrent.ExecutorService;
@@ -75,7 +75,7 @@ public final class ECBlockInputStreamFactoryImpl implements
    */
   public BlockExtendedInputStream create(boolean missingLocations,
       List<DatanodeDetails> failedLocations, ReplicationConfig repConfig,
-      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      BlockLocationInfo blockInfo, boolean verifyChecksum,
       XceiverClientFactory xceiverFactory,
       Function<BlockID, Pipeline> refreshFunction) {
     if (missingLocations) {
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
similarity index 97%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
index ecde9c66c8..49ee7c7538 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
+++ 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
@@ -23,8 +23,8 @@ 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.BlockLocationInfo;
 import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,7 +51,7 @@ public class ECBlockInputStreamProxy extends 
BlockExtendedInputStream {
   private final boolean verifyChecksum;
   private final XceiverClientFactory xceiverClientFactory;
   private final Function<BlockID, Pipeline> refreshFunction;
-  private final OmKeyLocationInfo blockInfo;
+  private final BlockLocationInfo blockInfo;
   private final ECBlockInputStreamFactory ecBlockInputStreamFactory;
 
   private BlockExtendedInputStream blockReader;
@@ -96,7 +96,7 @@ public class ECBlockInputStreamProxy extends 
BlockExtendedInputStream {
   }
 
   public ECBlockInputStreamProxy(ECReplicationConfig repConfig,
-      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      BlockLocationInfo blockInfo, boolean verifyChecksum,
       XceiverClientFactory xceiverClientFactory, Function<BlockID,
       Pipeline> refreshFunction, ECBlockInputStreamFactory streamFactory) {
     this.repConfig = repConfig;
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
similarity index 100%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
similarity index 99%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
index d5ec6db576..dc7daf8fa3 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
+++ 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
@@ -25,9 +25,9 @@ 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.BlockLocationInfo;
 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;
@@ -116,7 +116,7 @@ public class ECBlockReconstructedStripeInputStream extends 
ECBlockInputStream {
 
   @SuppressWarnings("checkstyle:ParameterNumber")
   public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
-      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      BlockLocationInfo blockInfo, boolean verifyChecksum,
       XceiverClientFactory xceiverClientFactory, Function<BlockID,
       Pipeline> refreshFunction, BlockInputStreamFactory streamFactory,
       ByteBufferPool byteBufferPool,
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
similarity index 100%
copy from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
copy to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/package-info.java
similarity index 61%
rename from 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
rename to 
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/package-info.java
index 956ed90b9f..493ece8074 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
+++ 
b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/package-info.java
@@ -15,29 +15,9 @@
  * 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;
+package org.apache.hadoop.ozone.client.io;
 
 /**
- * Exception thrown by EC Input Streams if there are not enough locations to
- * read the EC data successfully.
+ * This package contains Ozone I/O classes.
  */
-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/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java
similarity index 54%
copy from 
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
copy to 
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java
index 9df7518db4..286762d5d7 100644
--- 
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
+++ 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -14,14 +14,11 @@
  * License for the specific language governing permissions and limitations 
under
  * the License.
  */
-package org.apache.hadoop.ozone.om.helpers;
+package org.apache.hadoop.hdds.scm.storage;
 
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
-import org.apache.hadoop.hdds.scm.pipeline.UnknownPipelineStateException;
 import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
-import 
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation;
-import org.apache.hadoop.ozone.protocolPB.OzonePBHelper;
 import org.apache.hadoop.security.token.Token;
 
 import java.util.Objects;
@@ -30,9 +27,8 @@ import java.util.Objects;
  * One key can be too huge to fit in one container. In which case it gets split
  * into a number of subkeys. This class represents one such subkey instance.
  */
-public final class OmKeyLocationInfo {
+public class BlockLocationInfo {
   private final BlockID blockID;
-  // the id of this subkey in all the subkeys.
   private long length;
   private final long offset;
   // Block token, required for client authentication when security is enabled.
@@ -43,25 +39,16 @@ public final class OmKeyLocationInfo {
   private Pipeline pipeline;
 
   // PartNumber is set for Multipart upload Keys.
-  private int partNumber = -1;
-
-  private OmKeyLocationInfo(BlockID blockID, Pipeline pipeline, long length,
-                            long offset, int partNumber) {
-    this.blockID = blockID;
-    this.pipeline = pipeline;
-    this.length = length;
-    this.offset = offset;
-    this.partNumber = partNumber;
-  }
+  private int partNumber;
 
-  private OmKeyLocationInfo(BlockID blockID, Pipeline pipeline, long length,
-      long offset, Token<OzoneBlockTokenIdentifier> token, int partNumber) {
-    this.blockID = blockID;
-    this.pipeline = pipeline;
-    this.length = length;
-    this.offset = offset;
-    this.token = token;
-    this.partNumber = partNumber;
+  protected BlockLocationInfo(Builder builder) {
+    this.blockID = builder.blockID;
+    this.pipeline = builder.pipeline;
+    this.length = builder.length;
+    this.offset = builder.offset;
+    this.token = builder.token;
+    this.partNumber = builder.partNumber;
+    this.createVersion = builder.createVersion;
   }
 
   public void setCreateVersion(long version) {
@@ -125,7 +112,7 @@ public final class OmKeyLocationInfo {
   }
 
   /**
-   * Builder of OmKeyLocationInfo.
+   * Builder of BlockLocationInfo.
    */
   public static class Builder {
     private BlockID blockID;
@@ -134,13 +121,13 @@ public final class OmKeyLocationInfo {
     private Token<OzoneBlockTokenIdentifier> token;
     private Pipeline pipeline;
     private int partNumber;
+    private long createVersion;
 
     public Builder setBlockID(BlockID blockId) {
       this.blockID = blockId;
       return this;
     }
 
-    @SuppressWarnings("checkstyle:hiddenfield")
     public Builder setPipeline(Pipeline pipeline) {
       this.pipeline = pipeline;
       return this;
@@ -166,71 +153,14 @@ public final class OmKeyLocationInfo {
       return this;
     }
 
-    public OmKeyLocationInfo build() {
-      return new OmKeyLocationInfo(blockID, pipeline, length, offset, token,
-          partNumber);
-    }
-  }
-
-  public KeyLocation getProtobuf(int clientVersion) {
-    return getProtobuf(false, clientVersion);
-  }
-
-  public KeyLocation getProtobuf(boolean ignorePipeline, int clientVersion) {
-    KeyLocation.Builder builder = KeyLocation.newBuilder()
-        .setBlockID(blockID.getProtobuf())
-        .setLength(length)
-        .setOffset(offset)
-        .setCreateVersion(createVersion).setPartNumber(partNumber);
-    if (!ignorePipeline) {
-      try {
-        if (this.token != null) {
-          builder.setToken(OzonePBHelper.protoFromToken(token));
-        }
-
-        // Pipeline can be null when key create with override and
-        // on a versioning enabled bucket. for older versions of blocks
-        // We do not need to return pipeline as part of createKey,
-        // so we do not refresh pipeline in createKey, because of this reason
-        // for older version of blocks pipeline can be null.
-        // And also for key create we never need to return pipeline info
-        // for older version of blocks irrespective of versioning.
-
-        // Currently, we do not completely support bucket versioning.
-        // TODO: this needs to be revisited when bucket versioning
-        //  implementation is handled.
-
-        if (this.pipeline != null) {
-          builder.setPipeline(pipeline.getProtobufMessage(clientVersion));
-        }
-      } catch (UnknownPipelineStateException e) {
-        //TODO: fix me: we should not return KeyLocation without pipeline.
-      }
-    }
-    return builder.build();
-  }
-
-  private static Pipeline getPipeline(KeyLocation keyLocation) {
-    try {
-      return keyLocation.hasPipeline() ?
-          Pipeline.getFromProtobuf(keyLocation.getPipeline()) : null;
-    } catch (UnknownPipelineStateException e) {
-      return null;
+    public Builder setCreateVersion(long version) {
+      this.createVersion = version;
+      return this;
     }
-  }
 
-  public static OmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) {
-    OmKeyLocationInfo info = new OmKeyLocationInfo(
-        BlockID.getFromProtobuf(keyLocation.getBlockID()),
-        getPipeline(keyLocation),
-        keyLocation.getLength(),
-        keyLocation.getOffset(), keyLocation.getPartNumber());
-    if (keyLocation.hasToken()) {
-      info.token = (Token<OzoneBlockTokenIdentifier>)
-              OzonePBHelper.tokenFromProto(keyLocation.getToken());
+    public BlockLocationInfo build() {
+      return new BlockLocationInfo(this);
     }
-    info.setCreateVersion(keyLocation.getCreateVersion());
-    return info;
   }
 
   @Override
@@ -241,7 +171,8 @@ public final class OmKeyLocationInfo {
         ", offset=" + offset +
         ", token=" + token +
         ", pipeline=" + pipeline +
-        ", createVersion=" + createVersion  + ", partNumber=" + partNumber
+        ", createVersion=" + createVersion +
+        ", partNumber=" + partNumber
         + '}';
   }
 
@@ -253,7 +184,7 @@ public final class OmKeyLocationInfo {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    OmKeyLocationInfo that = (OmKeyLocationInfo) o;
+    BlockLocationInfo that = (BlockLocationInfo) o;
     return length == that.length &&
         offset == that.offset &&
         createVersion == that.createVersion &&
diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml
index 35248b7167..d981032018 100644
--- a/hadoop-hdds/pom.xml
+++ b/hadoop-hdds/pom.xml
@@ -107,6 +107,12 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd";>
         <version>${hdds.version}</version>
       </dependency>
 
+      <dependency>
+        <groupId>org.apache.ozone</groupId>
+        <artifactId>hdds-erasurecode</artifactId>
+        <version>${hdds.version}</version>
+      </dependency>
+
       <dependency>
         <groupId>org.apache.ozone</groupId>
         <artifactId>hdds-client</artifactId>
diff --git 
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
 
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
index 9df7518db4..9ff1ca94ea 100644
--- 
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
+++ 
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
@@ -19,156 +19,72 @@ package org.apache.hadoop.ozone.om.helpers;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.UnknownPipelineStateException;
+import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
 import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import 
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation;
 import org.apache.hadoop.ozone.protocolPB.OzonePBHelper;
 import org.apache.hadoop.security.token.Token;
 
-import java.util.Objects;
-
 /**
  * One key can be too huge to fit in one container. In which case it gets split
  * into a number of subkeys. This class represents one such subkey instance.
  */
-public final class OmKeyLocationInfo {
-  private final BlockID blockID;
-  // the id of this subkey in all the subkeys.
-  private long length;
-  private final long offset;
-  // Block token, required for client authentication when security is enabled.
-  private Token<OzoneBlockTokenIdentifier> token;
-  // the version number indicating when this block was added
-  private long createVersion;
-
-  private Pipeline pipeline;
-
-  // PartNumber is set for Multipart upload Keys.
-  private int partNumber = -1;
-
-  private OmKeyLocationInfo(BlockID blockID, Pipeline pipeline, long length,
-                            long offset, int partNumber) {
-    this.blockID = blockID;
-    this.pipeline = pipeline;
-    this.length = length;
-    this.offset = offset;
-    this.partNumber = partNumber;
-  }
-
-  private OmKeyLocationInfo(BlockID blockID, Pipeline pipeline, long length,
-      long offset, Token<OzoneBlockTokenIdentifier> token, int partNumber) {
-    this.blockID = blockID;
-    this.pipeline = pipeline;
-    this.length = length;
-    this.offset = offset;
-    this.token = token;
-    this.partNumber = partNumber;
-  }
-
-  public void setCreateVersion(long version) {
-    createVersion = version;
-  }
-
-  public long getCreateVersion() {
-    return createVersion;
-  }
-
-  public BlockID getBlockID() {
-    return blockID;
-  }
-
-  public long getContainerID() {
-    return blockID.getContainerID();
-  }
-
-  public long getLocalID() {
-    return blockID.getLocalID();
-  }
-
-  public Pipeline getPipeline() {
-    return pipeline;
-  }
-
-  public long getLength() {
-    return length;
-  }
-
-  public void setLength(long length) {
-    this.length = length;
-  }
+public final class OmKeyLocationInfo extends BlockLocationInfo {
 
-  public long getOffset() {
-    return offset;
-  }
-
-  public long getBlockCommitSequenceId() {
-    return blockID.getBlockCommitSequenceId();
-  }
-
-  public Token<OzoneBlockTokenIdentifier> getToken() {
-    return token;
-  }
-
-  public void setToken(Token<OzoneBlockTokenIdentifier> token) {
-    this.token = token;
-  }
-
-  public void setPipeline(Pipeline pipeline) {
-    this.pipeline = pipeline;
-  }
-
-  public void setPartNumber(int partNumber) {
-    this.partNumber = partNumber;
-  }
-
-  public int getPartNumber() {
-    return partNumber;
+  private OmKeyLocationInfo(Builder builder) {
+    super(builder);
   }
 
   /**
    * Builder of OmKeyLocationInfo.
    */
-  public static class Builder {
-    private BlockID blockID;
-    private long length;
-    private long offset;
-    private Token<OzoneBlockTokenIdentifier> token;
-    private Pipeline pipeline;
-    private int partNumber;
+  public static class Builder extends BlockLocationInfo.Builder {
 
-    public Builder setBlockID(BlockID blockId) {
-      this.blockID = blockId;
+    @Override
+    public Builder setBlockID(BlockID blockID) {
+      super.setBlockID(blockID);
       return this;
     }
 
-    @SuppressWarnings("checkstyle:hiddenfield")
+    @Override
     public Builder setPipeline(Pipeline pipeline) {
-      this.pipeline = pipeline;
+      super.setPipeline(pipeline);
       return this;
     }
 
-    public Builder setLength(long len) {
-      this.length = len;
+    @Override
+    public Builder setLength(long length) {
+      super.setLength(length);
       return this;
     }
 
-    public Builder setOffset(long off) {
-      this.offset = off;
+    @Override
+    public Builder setOffset(long offset) {
+      super.setOffset(offset);
       return this;
     }
 
-    public Builder setToken(Token<OzoneBlockTokenIdentifier> bToken) {
-      this.token = bToken;
+    @Override
+    public Builder setToken(Token<OzoneBlockTokenIdentifier> token) {
+      super.setToken(token);
       return this;
     }
 
-    public Builder setPartNumber(int partNum) {
-      this.partNumber = partNum;
+    @Override
+    public Builder setPartNumber(int partNumber) {
+      super.setPartNumber(partNumber);
       return this;
     }
 
+    @Override
+    public Builder setCreateVersion(long version) {
+      super.setCreateVersion(version);
+      return this;
+    }
+
+    @Override
     public OmKeyLocationInfo build() {
-      return new OmKeyLocationInfo(blockID, pipeline, length, offset, token,
-          partNumber);
+      return new OmKeyLocationInfo(this);
     }
   }
 
@@ -178,13 +94,15 @@ public final class OmKeyLocationInfo {
 
   public KeyLocation getProtobuf(boolean ignorePipeline, int clientVersion) {
     KeyLocation.Builder builder = KeyLocation.newBuilder()
-        .setBlockID(blockID.getProtobuf())
-        .setLength(length)
-        .setOffset(offset)
-        .setCreateVersion(createVersion).setPartNumber(partNumber);
+        .setBlockID(getBlockID().getProtobuf())
+        .setLength(getLength())
+        .setOffset(getOffset())
+        .setCreateVersion(getCreateVersion())
+        .setPartNumber(getPartNumber());
     if (!ignorePipeline) {
       try {
-        if (this.token != null) {
+        Token<OzoneBlockTokenIdentifier> token = getToken();
+        if (token != null) {
           builder.setToken(OzonePBHelper.protoFromToken(token));
         }
 
@@ -200,7 +118,8 @@ public final class OmKeyLocationInfo {
         // TODO: this needs to be revisited when bucket versioning
         //  implementation is handled.
 
-        if (this.pipeline != null) {
+        Pipeline pipeline = getPipeline();
+        if (pipeline != null) {
           builder.setPipeline(pipeline.getProtobufMessage(clientVersion));
         }
       } catch (UnknownPipelineStateException e) {
@@ -220,51 +139,18 @@ public final class OmKeyLocationInfo {
   }
 
   public static OmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) {
-    OmKeyLocationInfo info = new OmKeyLocationInfo(
-        BlockID.getFromProtobuf(keyLocation.getBlockID()),
-        getPipeline(keyLocation),
-        keyLocation.getLength(),
-        keyLocation.getOffset(), keyLocation.getPartNumber());
+    Builder builder = new Builder()
+        .setBlockID(BlockID.getFromProtobuf(keyLocation.getBlockID()))
+        .setLength(keyLocation.getLength())
+        .setOffset(keyLocation.getOffset())
+        .setPipeline(getPipeline(keyLocation))
+        .setCreateVersion(keyLocation.getCreateVersion())
+        .setPartNumber(keyLocation.getPartNumber());
     if (keyLocation.hasToken()) {
-      info.token = (Token<OzoneBlockTokenIdentifier>)
-              OzonePBHelper.tokenFromProto(keyLocation.getToken());
+      builder.setToken((Token<OzoneBlockTokenIdentifier>)
+          OzonePBHelper.tokenFromProto(keyLocation.getToken()));
     }
-    info.setCreateVersion(keyLocation.getCreateVersion());
-    return info;
-  }
-
-  @Override
-  public String  toString() {
-    return "{blockID={containerID=" + blockID.getContainerID() +
-        ", localID=" + blockID.getLocalID() + "}" +
-        ", length=" + length +
-        ", offset=" + offset +
-        ", token=" + token +
-        ", pipeline=" + pipeline +
-        ", createVersion=" + createVersion  + ", partNumber=" + partNumber
-        + '}';
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    OmKeyLocationInfo that = (OmKeyLocationInfo) o;
-    return length == that.length &&
-        offset == that.offset &&
-        createVersion == that.createVersion &&
-        Objects.equals(blockID, that.blockID) &&
-        Objects.equals(token, that.token) &&
-        Objects.equals(pipeline, that.pipeline);
+    return builder.build();
   }
 
-  @Override
-  public int hashCode() {
-    return Objects.hash(blockID, length, offset, token, createVersion,
-        pipeline);
-  }
 }
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
index 0503dbe759..f3649eabaf 100644
--- 
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
@@ -26,10 +26,10 @@ 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.BlockLocationInfo;
 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;
@@ -57,7 +57,7 @@ public final class ECStreamTestUtil {
   private ECStreamTestUtil() {
   }
 
-  public static OmKeyLocationInfo createKeyInfo(ReplicationConfig repConf,
+  public static BlockLocationInfo createKeyInfo(ReplicationConfig repConf,
       long blockLength, Map<DatanodeDetails, Integer> dnMap) {
 
     Pipeline pipeline = Pipeline.newBuilder()
@@ -68,7 +68,7 @@ public final class ECStreamTestUtil {
         .setReplicationConfig(repConf)
         .build();
 
-    OmKeyLocationInfo keyInfo = new OmKeyLocationInfo.Builder()
+    BlockLocationInfo keyInfo = new BlockLocationInfo.Builder()
         .setBlockID(new BlockID(1, 1))
         .setLength(blockLength)
         .setOffset(0)
@@ -78,7 +78,7 @@ public final class ECStreamTestUtil {
     return keyInfo;
   }
 
-  public static OmKeyLocationInfo createKeyInfo(ReplicationConfig repConf,
+  public static BlockLocationInfo createKeyInfo(ReplicationConfig repConf,
       int nodeCount, long blockLength) {
     Map<DatanodeDetails, Integer> datanodes = new HashMap<>();
     for (int i = 0; i < nodeCount; i++) {
@@ -257,7 +257,7 @@ public final class ECStreamTestUtil {
 
     public synchronized BlockExtendedInputStream create(
         ReplicationConfig repConfig,
-        OmKeyLocationInfo blockInfo, Pipeline pipeline,
+        BlockLocationInfo blockInfo, Pipeline pipeline,
         Token<OzoneBlockTokenIdentifier> token, boolean verifyChecksum,
         XceiverClientFactory xceiverFactory,
         Function<BlockID, Pipeline> refreshFunction) {
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
index 17a0a6f47e..a9f9407635 100644
--- 
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
@@ -25,12 +25,12 @@ 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.BlockLocationInfo;
 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;
@@ -65,7 +65,7 @@ public class TestECBlockInputStream {
   @Test
   public void testSufficientLocations() {
     // EC-3-2, 5MB block, so all 3 data locations are needed
-    OmKeyLocationInfo keyInfo = ECStreamTestUtil
+    BlockLocationInfo keyInfo = ECStreamTestUtil
         .createKeyInfo(repConfig, 5, 5 * ONEMB);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
@@ -116,7 +116,7 @@ public class TestECBlockInputStream {
   public void testCorrectBlockSizePassedToBlockStreamLessThanCell()
       throws IOException {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB - 100);
 
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
@@ -133,7 +133,7 @@ public class TestECBlockInputStream {
   public void testCorrectBlockSizePassedToBlockStreamTwoCells()
       throws IOException {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB + 100);
 
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
@@ -149,7 +149,7 @@ public class TestECBlockInputStream {
   public void testCorrectBlockSizePassedToBlockStreamThreeCells()
       throws IOException {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 2 * ONEMB + 100);
 
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
@@ -166,7 +166,7 @@ public class TestECBlockInputStream {
   public void 
testCorrectBlockSizePassedToBlockStreamThreeFullAndPartialStripe()
       throws IOException {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 10 * ONEMB + 100);
 
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
@@ -183,7 +183,7 @@ public class TestECBlockInputStream {
   public void testCorrectBlockSizePassedToBlockStreamSingleFullCell()
       throws IOException {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB);
 
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
@@ -198,7 +198,7 @@ public class TestECBlockInputStream {
   public void testCorrectBlockSizePassedToBlockStreamSeveralFullCells()
       throws IOException {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 9 * ONEMB);
 
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
@@ -213,7 +213,7 @@ public class TestECBlockInputStream {
 
   @Test
   public void testSimpleRead() throws IOException {
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
@@ -236,7 +236,7 @@ public class TestECBlockInputStream {
    */
   @Test
   public void testSimpleReadUnderOneChunk() throws IOException {
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 1, ONEMB);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
@@ -255,7 +255,7 @@ public class TestECBlockInputStream {
 
   @Test
   public void testReadPastEOF() throws IOException {
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 50);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
@@ -274,7 +274,7 @@ public class TestECBlockInputStream {
     // EC-3-2, 5MB block, so all 3 data locations are needed
     repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
         100);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
@@ -309,7 +309,7 @@ public class TestECBlockInputStream {
   public void testSeekPastBlockLength() throws IOException {
     repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
         ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 100);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
@@ -321,7 +321,7 @@ public class TestECBlockInputStream {
   public void testSeekToLength() throws IOException {
     repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
         ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 100);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
@@ -333,7 +333,7 @@ public class TestECBlockInputStream {
   public void testSeekToLengthZeroLengthBlock() throws IOException {
     repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
         ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 0);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
@@ -347,7 +347,7 @@ public class TestECBlockInputStream {
   public void testSeekToValidPosition() throws IOException {
     repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
         ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
@@ -376,7 +376,7 @@ public class TestECBlockInputStream {
   public void testErrorReadingBlockReportsBadLocation() throws IOException {
     repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
         ONEMB);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
     try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
@@ -417,7 +417,7 @@ public class TestECBlockInputStream {
     }
 
     public synchronized BlockExtendedInputStream create(
-        ReplicationConfig repConfig, OmKeyLocationInfo blockInfo,
+        ReplicationConfig repConfig, BlockLocationInfo blockInfo,
         Pipeline pipeline, Token<OzoneBlockTokenIdentifier> token,
         boolean verifyChecksum, XceiverClientFactory xceiverFactory,
         Function<BlockID, Pipeline> refreshFunction) {
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
index fd4e8add2e..a45ab43c84 100644
--- 
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
@@ -24,10 +24,10 @@ 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.BlockLocationInfo;
 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;
@@ -93,7 +93,7 @@ public class TestECBlockInputStreamProxy {
   public void testAvailableDataLocations() {
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo blockInfo =
+    BlockLocationInfo blockInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, 1024, dnMap);
     Assert.assertEquals(1, ECBlockInputStreamProxy.availableDataLocations(
         blockInfo.getPipeline(), 1));
@@ -120,7 +120,7 @@ public class TestECBlockInputStreamProxy {
 
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo blockInfo =
+    BlockLocationInfo blockInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
 
     try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
@@ -135,7 +135,7 @@ public class TestECBlockInputStreamProxy {
 
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo blockInfo =
+    BlockLocationInfo blockInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
 
     try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
@@ -150,7 +150,7 @@ public class TestECBlockInputStreamProxy {
 
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo blockInfo =
+    BlockLocationInfo blockInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
 
     dataGenerator = new SplittableRandom(randomSeed);
@@ -172,7 +172,7 @@ public class TestECBlockInputStreamProxy {
 
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo blockInfo =
+    BlockLocationInfo blockInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
 
     try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
@@ -203,7 +203,7 @@ public class TestECBlockInputStreamProxy {
 
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo blockInfo =
+    BlockLocationInfo blockInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
 
     ByteBuffer readBuffer = ByteBuffer.allocate(100);
@@ -231,7 +231,7 @@ public class TestECBlockInputStreamProxy {
 
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(2, 3, 4, 5);
-    OmKeyLocationInfo blockInfo =
+    BlockLocationInfo blockInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
 
     ByteBuffer readBuffer = ByteBuffer.allocate(100);
@@ -259,7 +259,7 @@ public class TestECBlockInputStreamProxy {
 
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo blockInfo =
+    BlockLocationInfo blockInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
 
     ByteBuffer readBuffer = ByteBuffer.allocate(100);
@@ -301,7 +301,7 @@ public class TestECBlockInputStreamProxy {
 
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo blockInfo =
+    BlockLocationInfo blockInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
 
     ByteBuffer readBuffer = ByteBuffer.allocate(100);
@@ -352,7 +352,7 @@ public class TestECBlockInputStreamProxy {
   }
 
   private ECBlockInputStreamProxy createBISProxy(ECReplicationConfig rConfig,
-      OmKeyLocationInfo blockInfo) {
+      BlockLocationInfo blockInfo) {
     return new ECBlockInputStreamProxy(
         rConfig, blockInfo, true, null, null, streamFactory);
   }
@@ -382,7 +382,7 @@ public class TestECBlockInputStreamProxy {
     @Override
     public BlockExtendedInputStream create(boolean missingLocations,
         List<DatanodeDetails> failedDatanodes,
-        ReplicationConfig repConfig, OmKeyLocationInfo blockInfo,
+        ReplicationConfig repConfig, BlockLocationInfo blockInfo,
         boolean verifyChecksum, XceiverClientFactory xceiverFactory,
         Function<BlockID, Pipeline> refreshFunction) {
       this.failedLocations = failedDatanodes;
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
index ee3f50a55f..823ddd5850 100644
--- 
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
@@ -20,11 +20,11 @@ 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.hdds.scm.storage.BlockLocationInfo;
 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;
@@ -72,7 +72,7 @@ public class TestECBlockReconstructedInputStream {
 
   private ECBlockReconstructedStripeInputStream createStripeInputStream(
       Map<DatanodeDetails, Integer> dnMap, long blockLength) {
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
     streamFactory.setCurrentPipeline(keyInfo.getPipeline());
     return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
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
index 0eff8f9ab1..b111b1208a 100644
--- 
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
@@ -19,6 +19,7 @@ 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.hdds.scm.storage.BlockLocationInfo;
 import org.apache.hadoop.io.ByteBufferPool;
 import org.apache.hadoop.io.ElasticByteBufferPool;
 import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
@@ -26,7 +27,6 @@ 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;
@@ -82,7 +82,7 @@ public class TestECBlockReconstructedStripeInputStream {
   @Test
   public void testSufficientLocations() throws IOException {
     // One chunk, only 1 location.
-    OmKeyLocationInfo keyInfo = ECStreamTestUtil
+    BlockLocationInfo keyInfo = ECStreamTestUtil
         .createKeyInfo(repConfig, 1, ONEMB);
     try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
       Assert.assertTrue(ecb.hasSufficientLocations());
@@ -168,7 +168,7 @@ public class TestECBlockReconstructedStripeInputStream {
       streamFactory = new TestBlockInputStreamFactory();
       addDataStreamsToFactory(dataBufs, parity);
 
-      OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+      BlockLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
           stripeSize() * 3 + partialStripeSize, dnMap);
       streamFactory.setCurrentPipeline(keyInfo.getPipeline());
 
@@ -223,7 +223,7 @@ public class TestECBlockReconstructedStripeInputStream {
     // from the parity and padded blocks 2 and 3.
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(4, 5);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
     streamFactory.setCurrentPipeline(keyInfo.getPipeline());
     dataGen = new SplittableRandom(randomSeed);
@@ -265,7 +265,7 @@ public class TestECBlockReconstructedStripeInputStream {
     // from the parity and padded blocks 2 and 3.
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(4, 5);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
     streamFactory.setCurrentPipeline(keyInfo.getPipeline());
     dataGen = new SplittableRandom(randomSeed);
@@ -322,7 +322,7 @@ public class TestECBlockReconstructedStripeInputStream {
       addDataStreamsToFactory(dataBufs, parity);
       ByteBuffer[] bufs = allocateByteBuffers(repConfig);
 
-      OmKeyLocationInfo keyInfo =
+      BlockLocationInfo keyInfo =
           ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
       streamFactory.setCurrentPipeline(keyInfo.getPipeline());
       dataGen = new SplittableRandom(randomSeed);
@@ -365,7 +365,7 @@ public class TestECBlockReconstructedStripeInputStream {
     // from the parity and padded blocks 2 and 3.
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(4, 5);
-    OmKeyLocationInfo keyInfo =
+    BlockLocationInfo keyInfo =
         ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
     streamFactory.setCurrentPipeline(keyInfo.getPipeline());
     try (ECBlockReconstructedStripeInputStream ecb =
@@ -407,7 +407,7 @@ public class TestECBlockReconstructedStripeInputStream {
       streamFactory = new TestBlockInputStreamFactory();
       addDataStreamsToFactory(dataBufs, parity);
 
-      OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+      BlockLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
           stripeSize() * 3 + partialStripeSize, dnMap);
       streamFactory.setCurrentPipeline(keyInfo.getPipeline());
 
@@ -461,7 +461,7 @@ public class TestECBlockReconstructedStripeInputStream {
   public void testSeekToPartialOffsetFails() {
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 4, 5);
-    OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+    BlockLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
         stripeSize() * 3, dnMap);
     streamFactory.setCurrentPipeline(keyInfo.getPipeline());
 
@@ -503,7 +503,7 @@ public class TestECBlockReconstructedStripeInputStream {
       // 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,
+      BlockLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
           stripeSize() * 3 + partialStripeSize, dnMap);
       streamFactory.setCurrentPipeline(keyInfo.getPipeline());
 
@@ -583,7 +583,7 @@ public class TestECBlockReconstructedStripeInputStream {
     // when containers are reported by SCM.
     Map<DatanodeDetails, Integer> dnMap =
           ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+    BlockLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
         blockLength, dnMap);
     streamFactory.setCurrentPipeline(keyInfo.getPipeline());
 
@@ -609,7 +609,7 @@ public class TestECBlockReconstructedStripeInputStream {
 
     Map<DatanodeDetails, Integer> dnMap =
         ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
-    OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+    BlockLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
         stripeSize() * 3 + partialStripeSize, dnMap);
     streamFactory.setCurrentPipeline(keyInfo.getPipeline());
 
@@ -643,7 +643,7 @@ public class TestECBlockReconstructedStripeInputStream {
   }
 
   private ECBlockReconstructedStripeInputStream createInputStream(
-      OmKeyLocationInfo keyInfo) {
+      BlockLocationInfo keyInfo) {
     return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
         null, null, streamFactory, bufferPool, ecReconstructExecutor);
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to