You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by so...@apache.org on 2021/10/04 16:39:48 UTC

[ozone] branch HDDS-3816-ec updated: HDDS-5741. EC: Remove hard coded chunksize and get from from ReplicationConfig (#2687)

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

sodonnell pushed a commit to branch HDDS-3816-ec
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/HDDS-3816-ec by this push:
     new 5cf2516  HDDS-5741. EC: Remove hard coded chunksize and get from from ReplicationConfig (#2687)
5cf2516 is described below

commit 5cf251654e5bc143c67aab812582cc12f0784fb1
Author: Stephen O'Donnell <st...@gmail.com>
AuthorDate: Mon Oct 4 17:39:32 2021 +0100

    HDDS-5741. EC: Remove hard coded chunksize and get from from ReplicationConfig (#2687)
---
 .../hadoop/hdds/client/ECReplicationConfig.java    |  5 +++
 .../hadoop/ozone/client/io/ECBlockInputStream.java | 16 +--------
 .../hadoop/ozone/client/io/ECKeyOutputStream.java  | 22 ++++++-------
 .../apache/hadoop/ozone/client/rpc/RpcClient.java  |  4 ++-
 .../hadoop/ozone/client/TestOzoneClient.java       |  7 ++--
 .../hadoop/ozone/client/TestOzoneECClient.java     | 18 ++++++----
 .../ozone/client/rpc/TestECKeyOutputStream.java    | 12 ++++---
 .../client/rpc/read/TestECBlockInputStream.java    | 38 +++++++++++-----------
 8 files changed, 63 insertions(+), 59 deletions(-)

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
index ca0b520..93cef00 100644
--- 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
@@ -37,6 +37,11 @@ public class ECReplicationConfig implements ReplicationConfig {
   public enum EcCodec {
     RS, XOR;
 
+    @Override
+    public String toString() {
+      return name().toLowerCase();
+    }
+
     public static String allValuesAsString() {
       return EnumSet.allOf(EcCodec.class)
           .stream()
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
index f919528..a84452d 100644
--- 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
@@ -47,8 +47,6 @@ public class ECBlockInputStream extends BlockExtendedInputStream {
       LoggerFactory.getLogger(ECBlockInputStream.class);
 
   private final ECReplicationConfig repConfig;
-  // TODO - HDDS-5741 - remove hardcoded value
-  private static final int HARDCODED_CHUNK_SIZE = 1024;
   private final int ecChunkSize;
   private final BlockInputStreamFactory streamFactory;
   private final boolean verifyChecksum;
@@ -67,20 +65,8 @@ public class ECBlockInputStream extends BlockExtendedInputStream {
       OmKeyLocationInfo blockInfo, boolean verifyChecksum,
       XceiverClientFactory xceiverClientFactory, Function<BlockID,
       Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
-
-    this(repConfig, HARDCODED_CHUNK_SIZE, blockInfo, verifyChecksum,
-        xceiverClientFactory, refreshFunction, streamFactory);
-  }
-
-  // TODO - HDDS-5741 - remove this constructor - ecChunkSize should not be
-  //        there
-  public ECBlockInputStream(ECReplicationConfig repConfig, int ecChunkSize,
-      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
-      XceiverClientFactory xceiverClientFactory, Function<BlockID,
-      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
     this.repConfig = repConfig;
-    // TODO - HDDS-5741
-    this.ecChunkSize = ecChunkSize;
+    this.ecChunkSize = repConfig.getEcChunkSize();
     this.verifyChecksum = verifyChecksum;
     this.blockInfo = blockInfo;
     this.streamFactory = streamFactory;
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
index 2f1f815..b5ab25c 100644
--- 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
@@ -26,7 +26,6 @@ import java.util.List;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 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.OzoneClientConfig;
@@ -60,14 +59,12 @@ import org.slf4j.LoggerFactory;
 public class ECKeyOutputStream extends KeyOutputStream {
   private OzoneClientConfig config;
   private ECChunkBuffers ecChunkBufferCache;
-  private int ecChunkSize = 1024;
+  private int ecChunkSize;
   private final int numDataBlks;
   private final int numParityBlks;
   private static final ByteBufferPool BUFFER_POOL = new ElasticByteBufferPool();
   private final RawErasureEncoder encoder;
-  // TODO: EC: Currently using the below EC Schema. This has to be modified and
-  //  created dynamically once OM return the configured scheme details.
-  private static final String DEFAULT_CODEC_NAME = "rs";
+  private final ECReplicationConfig.EcCodec ecCodec;
 
   private long currentBlockGroupLen = 0;
   /**
@@ -109,17 +106,18 @@ public class ECKeyOutputStream extends KeyOutputStream {
   @SuppressWarnings({"parameternumber", "squid:S00107"})
   public ECKeyOutputStream(OzoneClientConfig config, OpenKeySession handler,
       XceiverClientFactory xceiverClientManager, OzoneManagerProtocol omClient,
-      int chunkSize, String requestId, ReplicationConfig replicationConfig,
+      int chunkSize, String requestId, ECReplicationConfig replicationConfig,
       String uploadID, int partNumber, boolean isMultipart,
       boolean unsafeByteBufferConversion) {
     this.config = config;
     // For EC, cell/chunk size and buffer size can be same for now.
+    ecChunkSize = replicationConfig.getEcChunkSize();
+    this.ecCodec = replicationConfig.getCodec();
     this.config.setStreamBufferMaxSize(ecChunkSize);
     this.config.setStreamBufferFlushSize(ecChunkSize);
     this.config.setStreamBufferSize(ecChunkSize);
-    assert replicationConfig instanceof ECReplicationConfig;
-    this.numDataBlks = ((ECReplicationConfig) replicationConfig).getData();
-    this.numParityBlks = ((ECReplicationConfig) replicationConfig).getParity();
+    this.numDataBlks = replicationConfig.getData();
+    this.numParityBlks = replicationConfig.getParity();
     ecChunkBufferCache =
         new ECChunkBuffers(ecChunkSize, numDataBlks, numParityBlks);
     OmKeyInfo info = handler.getKeyInfo();
@@ -135,7 +133,7 @@ public class ECKeyOutputStream extends KeyOutputStream {
     this.writeOffset = 0;
     OzoneConfiguration conf = new OzoneConfiguration();
     ECSchema schema =
-        new ECSchema(DEFAULT_CODEC_NAME, numDataBlks, numParityBlks);
+        new ECSchema(ecCodec.toString(), numDataBlks, numParityBlks);
     ErasureCodecOptions options = new ErasureCodecOptions(schema);
     RSErasureCodec codec = new RSErasureCodec(conf, options);
     this.encoder = CodecUtil.createRawEncoder(conf,
@@ -576,7 +574,7 @@ public class ECKeyOutputStream extends KeyOutputStream {
     private boolean isMultipartKey;
     private boolean unsafeByteBufferConversion;
     private OzoneClientConfig clientConfig;
-    private ReplicationConfig replicationConfig;
+    private ECReplicationConfig replicationConfig;
 
     public Builder setMultipartUploadID(String uploadID) {
       this.multipartUploadID = uploadID;
@@ -629,7 +627,7 @@ public class ECKeyOutputStream extends KeyOutputStream {
     }
 
     public ECKeyOutputStream.Builder setReplicationConfig(
-        ReplicationConfig replConfig) {
+        ECReplicationConfig replConfig) {
       this.replicationConfig = replConfig;
       return this;
     }
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 0d79676..3647538 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
@@ -44,6 +44,7 @@ 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;
@@ -1408,7 +1409,8 @@ public class RpcClient implements ClientProtocol {
       keyOutputStream = new ECKeyOutputStream.Builder().setHandler(openKey)
           .setXceiverClientManager(xceiverClientManager)
           .setOmClient(ozoneManagerClient).setRequestID(requestId)
-          .setReplicationConfig(openKey.getKeyInfo().getReplicationConfig())
+          .setReplicationConfig(
+              (ECReplicationConfig)openKey.getKeyInfo().getReplicationConfig())
           .enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
           .setConfig(clientConfig).build();
     } else {
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 804831a..4dc47c3 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
@@ -193,15 +193,18 @@ public class TestOzoneClient {
     ConfigurationSource config = new InMemoryConfiguration();
     int data = 3;
     int parity = 2;
+    int chunkSize = 1024;
     createNewClient(config, new MultiNodePipelineBlockAllocator(data + parity));
-    String value = new String(new byte[1024], UTF_8);
+    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), new HashMap<>())) {
+              new ECReplicationConfig(data, parity,
+                  ECReplicationConfig.EcCodec.RS, chunkSize),
+              new HashMap<>())) {
         out.write(value.getBytes(UTF_8));
         out.write(value.getBytes(UTF_8));
       }
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
index cf5872c..5fe11a3 100644
--- 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
@@ -198,7 +198,8 @@ public class TestOzoneECClient {
       throws IOException {
     final OzoneBucket bucket = writeIntoECKey(inputChunks, keyName,
         new DefaultReplicationConfig(ReplicationType.EC,
-            new ECReplicationConfig(dataBlocks, parityBlocks)));
+            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.
@@ -215,7 +216,8 @@ public class TestOzoneECClient {
       throws IOException {
     final OzoneBucket bucket = writeIntoECKey(inputChunks, keyName,
         new DefaultReplicationConfig(ReplicationType.EC,
-            new ECReplicationConfig(dataBlocks, parityBlocks)));
+            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.
@@ -241,7 +243,8 @@ public class TestOzoneECClient {
     OzoneBucket bucket = volume.getBucket(bucketName);
 
     try (OzoneOutputStream out = bucket.createKey(keyName, 2000,
-        new ECReplicationConfig(dataBlocks, parityBlocks), new HashMap<>())) {
+        new ECReplicationConfig(dataBlocks, parityBlocks,
+            ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
       for (int i = 0; i < inputChunks[0].length; i++) {
         out.write(inputChunks[0][i]);
       }
@@ -266,7 +269,8 @@ public class TestOzoneECClient {
     OzoneBucket bucket = volume.getBucket(bucketName);
 
     try (OzoneOutputStream out = bucket.createKey(keyName, 2000,
-        new ECReplicationConfig(dataBlocks, parityBlocks), new HashMap<>())) {
+        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]);
       }
@@ -297,7 +301,8 @@ public class TestOzoneECClient {
             inputChunks[inputChunks.length - 1].length - 1);
 
     try (OzoneOutputStream out = bucket.createKey(keyName, 2000,
-        new ECReplicationConfig(dataBlocks, parityBlocks), new HashMap<>())) {
+        new ECReplicationConfig(dataBlocks, parityBlocks,
+            ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
       for (int i = 0; i < inputChunks.length - 1; i++) {
         out.write(inputChunks[i]);
       }
@@ -335,7 +340,8 @@ public class TestOzoneECClient {
     OzoneBucket bucket = volume.getBucket(bucketName);
 
     try (OzoneOutputStream out = bucket.createKey(key, 2000,
-        new ECReplicationConfig(3, 2), new HashMap<>())) {
+        new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+            chunkSize), new HashMap<>())) {
       for (int i = 0; i < chunks.length; i++) {
         out.write(chunks[i]);
       }
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
index b8169ca..d6a0ea8 100644
--- 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
@@ -129,7 +129,8 @@ public class TestECKeyOutputStream {
   @Test
   public void testCreateKeyWithECReplicationConfig() throws Exception {
     try (OzoneOutputStream key = TestHelper
-        .createKey(keyString, new ECReplicationConfig(3, 2), 2000, objectStore,
+        .createKey(keyString, new ECReplicationConfig(3, 2,
+            ECReplicationConfig.EcCodec.RS, chunkSize), 2000, objectStore,
             volumeName, bucketName)) {
       Assert.assertTrue(key.getOutputStream() instanceof ECKeyOutputStream);
     }
@@ -154,7 +155,8 @@ public class TestECKeyOutputStream {
     final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
     bucketArgs.setDefaultReplicationConfig(
         new DefaultReplicationConfig(ReplicationType.EC,
-            new ECReplicationConfig(3, 2)));
+            new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+                chunkSize)));
 
     volume.createBucket(myBucket, bucketArgs.build());
     OzoneBucket bucket = volume.getBucket(myBucket);
@@ -182,7 +184,8 @@ public class TestECKeyOutputStream {
     final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
     bucketArgs.setDefaultReplicationConfig(
         new DefaultReplicationConfig(ReplicationType.EC,
-            new ECReplicationConfig(3, 2)));
+            new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+                chunkSize)));
 
     volume.createBucket(myBucket, bucketArgs.build());
     OzoneBucket bucket = volume.getBucket(myBucket);
@@ -204,7 +207,8 @@ public class TestECKeyOutputStream {
     int data = 3;
     int parity = 2;
     try (OzoneOutputStream key = TestHelper
-        .createKey(keyString, new ECReplicationConfig(data, parity), 1024,
+        .createKey(keyString, new ECReplicationConfig(data, parity,
+            ECReplicationConfig.EcCodec.RS, chunkSize), 1024,
             objectStore, volumeName, bucketName)) {
       final List<BlockOutputStreamEntry> streamEntries =
           ((ECKeyOutputStream) key.getOutputStream()).getStreamEntries();
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 9790843..647483a 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
@@ -34,7 +34,6 @@ 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.Ignore;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -57,7 +56,8 @@ public class TestECBlockInputStream {
 
   @Before
   public void setup() {
-    repConfig = new ECReplicationConfig(3, 2);
+    repConfig = new ECReplicationConfig(3, 2,
+        ECReplicationConfig.EcCodec.RS, ONEMB);
     streamFactory = new TestBlockInputStreamFactory();
   }
 
@@ -66,14 +66,14 @@ public class TestECBlockInputStream {
   public void testSufficientLocations() {
     // EC-3-2, 5MB block, so all 3 data locations are needed
     OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 5 * ONEMB);
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, 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 = createKeyInfo(repConfig, 5, 5000 * ONEMB);
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
       Assert.assertTrue(ecb.hasSufficientLocations());
     }
@@ -83,7 +83,7 @@ public class TestECBlockInputStream {
     // EC-3-2, 1 byte short of 1MB with 1 location
     dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
     keyInfo = createKeyInfo(repConfig, ONEMB - 1, dnMap);
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
       Assert.assertTrue(ecb.hasSufficientLocations());
     }
@@ -92,7 +92,7 @@ public class TestECBlockInputStream {
     dnMap.clear();
     dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 2);
     keyInfo = createKeyInfo(repConfig, ONEMB, dnMap);
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
       Assert.assertFalse(ecb.hasSufficientLocations());
     }
@@ -102,7 +102,7 @@ public class TestECBlockInputStream {
     dnMap.clear();
     dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
     keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
       Assert.assertFalse(ecb.hasSufficientLocations());
     }
@@ -114,7 +114,7 @@ public class TestECBlockInputStream {
     dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 4);
     dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 5);
     keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
       Assert.assertFalse(ecb.hasSufficientLocations());
     }
@@ -126,7 +126,7 @@ public class TestECBlockInputStream {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
     OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, ONEMB - 100);
 
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    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
@@ -142,7 +142,7 @@ public class TestECBlockInputStream {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
     OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, ONEMB + 100);
 
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB, 
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
       ecb.read(buf);
       List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
@@ -157,7 +157,7 @@ public class TestECBlockInputStream {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
     OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 2 * ONEMB + 100);
 
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
       ecb.read(buf);
       List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
@@ -173,7 +173,7 @@ public class TestECBlockInputStream {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
     OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 10 * ONEMB + 100);
 
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
       ecb.read(buf);
       List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
@@ -189,7 +189,7 @@ public class TestECBlockInputStream {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
     OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, ONEMB);
 
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
       ecb.read(buf);
       List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
@@ -203,7 +203,7 @@ public class TestECBlockInputStream {
     ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
     OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 9 * ONEMB);
 
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
       ecb.read(buf);
       List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
@@ -216,7 +216,7 @@ public class TestECBlockInputStream {
   @Test
   public void testSimpleRead() throws IOException {
     OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 5 * ONEMB);
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
 
       ByteBuffer buf = ByteBuffer.allocate(100);
@@ -234,7 +234,7 @@ public class TestECBlockInputStream {
   @Test
   public void testReadPastEOF() throws IOException {
     OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 50);
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
         keyInfo, true, null, null, streamFactory)) {
 
       ByteBuffer buf = ByteBuffer.allocate(100);
@@ -246,13 +246,13 @@ public class TestECBlockInputStream {
     }
   }
 
-  @Ignore("HDDS-5741")
-  // TODO - HDDS-5741 this test needs the RepConfig codec to be set correctly
   @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 = createKeyInfo(repConfig, 5, 5 * ONEMB);
-    try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig, 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,

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org