You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/09/16 16:02:58 UTC

[GitHub] [ozone] sodonnel opened a new pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

sodonnel opened a new pull request #2653:
URL: https://github.com/apache/ozone/pull/2653


   ## What changes were proposed in this pull request?
   
   Adapt KeyInputStream to read EC Blocks
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5550
   
   ## How was this patch tested?
   
   Modified and existing tests
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] sodonnel commented on a change in pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
sodonnel commented on a change in pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#discussion_r712356087



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
##########
@@ -19,60 +19,77 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.NotImplementedException;
-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.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.BlockInputStream;
-import org.apache.hadoop.hdds.scm.storage.ByteArrayReader;
-import org.apache.hadoop.hdds.scm.storage.ByteBufferReader;
+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.io.InputStream;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.function.Function;
 
 /**
  * Class to read data from an EC Block Group.
  */
-public class ECBlockInputStream extends InputStream
-    implements Seekable, CanUnbuffer, ByteBufferReadable {
+public class ECBlockInputStream extends BlockExtendedInputStream {
 
-  private static final int EOF = -1;
+  private static final Logger LOG =
+      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 BlockInputStreamProvider streamProvider;
+  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 DatanodeDetails[] parityLocations;
-  private final BlockInputStream[] blockStreams;
+  private final BlockExtendedInputStream[] blockStreams;
   private final int maxLocations;
 
-  private int position = 0;
+  private long position = 0;
   private boolean closed = false;
 
+  public ECBlockInputStream(ECReplicationConfig repConfig,
+      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,
-      BlockInputStreamProvider streamProvider) {
+      XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
     this.repConfig = repConfig;
+    // TODO - HDDS-5741
     this.ecChunkSize = ecChunkSize;
     this.verifyChecksum = verifyChecksum;
     this.blockInfo = blockInfo;
-    this.streamProvider = streamProvider;
+    this.streamFactory = streamFactory;
+    this.xceiverClientFactory = xceiverClientFactory;
+    this.refreshFunction = refreshFunction;
     this.maxLocations = repConfig.getData() + repConfig.getParity();
     this.dataLocations = new DatanodeDetails[repConfig.getData()];
     this.parityLocations = new DatanodeDetails[repConfig.getParity()];
-    this.blockStreams = new BlockInputStream[repConfig.getData()];
+    this.blockStreams = new BlockExtendedInputStream[repConfig.getData()];

Review comment:
       My plan is to split out the reading logic in ECBLockInputStream and delegate calls to “ECBlockInputStreamRecovery and ECBlockInputStreamHealthy (named something like that).
   
   So there will be a class to do a recovery block read and a class to do a straight read, which is really what we have now, but perhaps extracted into another class.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] umamaheswararao merged pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
umamaheswararao merged pull request #2653:
URL: https://github.com/apache/ozone/pull/2653


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] umamaheswararao edited a comment on pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
umamaheswararao edited a comment on pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#issuecomment-925010222


   > I don't want to make it bigger and confuse
   
   Just to be clarify: My proposed suggestion was not to make big. Its just matter to move the hard coded value from InputStream classes to ECReplConfig and use that. But it's not worth to wait this much for that change and blocked on that small change. :-) Let's move on to other JIRAs. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] sodonnel commented on a change in pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
sodonnel commented on a change in pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#discussion_r712469303



##########
File path: hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneECClient.java
##########
@@ -184,29 +184,12 @@ public void testPutECKeyAndReadContent() throws IOException {
     Assert.assertEquals(keyName, key.getName());
     try (OzoneInputStream is = bucket.readKey(keyName)) {
       byte[] fileContent = new byte[1024];
-      Assert.assertEquals(inputChunks[0].length, is.read(fileContent));
-      Assert.assertEquals(new String(inputChunks[0], UTF_8),
-          new String(fileContent, UTF_8));
-    }
-
-    // Since EC read is not ready yet, let's use the regular read by
-    // tweaking the pipeline.
-    // Remove first node in EC pipeline. So, regular read will hit the
-    // first node in pipeline and assert for second chunk in EC data.
-    updatePipelineToKeepSingleNode(2);
-    try (OzoneInputStream is = bucket.readKey(keyName)) {
-      byte[] fileContent = new byte[1024];
-      Assert.assertEquals(inputChunks[1].length, is.read(fileContent));
-      Assert.assertEquals(new String(inputChunks[1], UTF_8),
-          new String(fileContent, UTF_8));
-    }
-
-    updatePipelineToKeepSingleNode(3);
-    try (OzoneInputStream is = bucket.readKey(keyName)) {
-      byte[] fileContent = new byte[1024];
-      Assert.assertEquals(inputChunks[2].length, is.read(fileContent));
-      Assert.assertEquals(new String(inputChunks[2], UTF_8),
-          new String(fileContent, UTF_8));
+      for (int i=0; i<3; i++) {

Review comment:
       Good point. I have fixed this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] sodonnel commented on a change in pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
sodonnel commented on a change in pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#discussion_r712468165



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
##########
@@ -19,60 +19,77 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.NotImplementedException;
-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.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.BlockInputStream;
-import org.apache.hadoop.hdds.scm.storage.ByteArrayReader;
-import org.apache.hadoop.hdds.scm.storage.ByteBufferReader;
+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.io.InputStream;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.function.Function;
 
 /**
  * Class to read data from an EC Block Group.
  */
-public class ECBlockInputStream extends InputStream
-    implements Seekable, CanUnbuffer, ByteBufferReadable {
+public class ECBlockInputStream extends BlockExtendedInputStream {
 
-  private static final int EOF = -1;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockInputStream.class);
 
   private final ECReplicationConfig repConfig;
+  // TODO - HDDS-5741 - remove hardcoded value

Review comment:
       This is probably a good idea, otherwise we will missing things. I wonder if we should do that as part of the TODO jira I mentioned above, or just go ahead and add the bits to ECReplicationConfig as part of that Jira and get it done?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] umamaheswararao commented on pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#issuecomment-925006061


   Ok let me commit this. WE can do that other parts in new JIRA.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] sodonnel commented on a change in pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
sodonnel commented on a change in pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#discussion_r712459758



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 from the given parameters.
+   * @param blockId The blockID
+   * @param blockLen The Block Length
+   * @param pipeline The pipeline used to read from the block
+   * @param token The block access token
+   * @param verifyChecksum Whether to verify checksums or not
+   * @param xceiverFactory Factor to create the xceiver in the client
+   * @param refreshFunction Function to refresh the pipeline if needed.
+   * @return A BlockInputStream instance
+   */
+  BlockExtendedInputStream create(BlockID blockId, long blockLen,
+      Pipeline pipeline, Token<OzoneBlockTokenIdentifier> token,
+      boolean verifyChecksum, XceiverClientFactory xceiverFactory,
+      Function<BlockID, Pipeline> refreshFunction);
+
+  /**
+   * 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);
+

Review comment:
       Sorry, I thought I had removed this. It is removed now!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] sodonnel commented on a change in pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
sodonnel commented on a change in pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#discussion_r712466962



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.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.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.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+
+import java.util.function.Function;
+
+/**
+ * Factory class to create various BlockStream instances.
+ */
+public class BlockInputStreamFactoryImpl implements BlockInputStreamFactory {
+
+  public static BlockInputStreamFactory getInstance() {
+    return new BlockInputStreamFactoryImpl();
+  }
+
+  /**
+   * Create a new BlockInputStream from the given parameters.
+   * @param blockId The blockID
+   * @param blockLen The Block Length
+   * @param pipeline The pipeline used to read from the block
+   * @param token The block access token
+   * @param verifyChecksum Whether to verify checksums or not
+   * @param xceiverFactory Factor to create the xceiver in the client
+   * @param refreshFunction Function to refresh the pipeline if needed.
+   * @return A BlockInputStream instance
+   */

Review comment:
       I started to remove this extra API, but then I ran into a small problem in ECBlockInputStream - I cannot just pass the blockInfo it receives through to the factory, as the blockInfo.getLength is the length of the blockGroup, so I need to change it. So I went ahead and updated it to see if it looks better, which it probably does.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] sodonnel commented on a change in pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
sodonnel commented on a change in pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#discussion_r712468680



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
##########
@@ -19,60 +19,77 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.NotImplementedException;
-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.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.BlockInputStream;
-import org.apache.hadoop.hdds.scm.storage.ByteArrayReader;
-import org.apache.hadoop.hdds.scm.storage.ByteBufferReader;
+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.io.InputStream;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.function.Function;
 
 /**
  * Class to read data from an EC Block Group.
  */
-public class ECBlockInputStream extends InputStream
-    implements Seekable, CanUnbuffer, ByteBufferReadable {
+public class ECBlockInputStream extends BlockExtendedInputStream {
 
-  private static final int EOF = -1;
+  private static final Logger LOG =
+      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 BlockInputStreamProvider streamProvider;
+  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 DatanodeDetails[] parityLocations;
-  private final BlockInputStream[] blockStreams;
+  private final BlockExtendedInputStream[] blockStreams;
   private final int maxLocations;
 
-  private int position = 0;
+  private long position = 0;
   private boolean closed = false;
 
+  public ECBlockInputStream(ECReplicationConfig repConfig,
+      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,
-      BlockInputStreamProvider streamProvider) {
+      XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
     this.repConfig = repConfig;
+    // TODO - HDDS-5741
     this.ecChunkSize = ecChunkSize;
     this.verifyChecksum = verifyChecksum;
     this.blockInfo = blockInfo;
-    this.streamProvider = streamProvider;
+    this.streamFactory = streamFactory;
+    this.xceiverClientFactory = xceiverClientFactory;
+    this.refreshFunction = refreshFunction;
     this.maxLocations = repConfig.getData() + repConfig.getParity();
     this.dataLocations = new DatanodeDetails[repConfig.getData()];
     this.parityLocations = new DatanodeDetails[repConfig.getParity()];
-    this.blockStreams = new BlockInputStream[repConfig.getData()];
+    this.blockStreams = new BlockExtendedInputStream[repConfig.getData()];

Review comment:
       I'll also add that this patch does not consider recovery yet - its all happy path, but we will add that in followup jiras.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] sodonnel commented on pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
sodonnel commented on pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#issuecomment-924729701


   @umamaheswararao If you are happy, I think we should commit this one as it stands, and then create a new Jira to add the codec and chunksize to ECRepConfig, then a further Jira to undo all the hard coded pieces. That way we will have smaller easier to review patches - this one is pretty large already and I don't want to make it bigger and confuse it with the ECRepConfig change.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] umamaheswararao commented on a change in pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on a change in pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#discussion_r711878965



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 from the given parameters.
+   * @param blockId The blockID
+   * @param blockLen The Block Length
+   * @param pipeline The pipeline used to read from the block
+   * @param token The block access token
+   * @param verifyChecksum Whether to verify checksums or not
+   * @param xceiverFactory Factor to create the xceiver in the client
+   * @param refreshFunction Function to refresh the pipeline if needed.
+   * @return A BlockInputStream instance
+   */
+  BlockExtendedInputStream create(BlockID blockId, long blockLen,
+      Pipeline pipeline, Token<OzoneBlockTokenIdentifier> token,
+      boolean verifyChecksum, XceiverClientFactory xceiverFactory,
+      Function<BlockID, Pipeline> refreshFunction);
+
+  /**
+   * 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);
+

Review comment:
       You may want to removed below commented code?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.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.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.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+
+import java.util.function.Function;
+
+/**
+ * Factory class to create various BlockStream instances.
+ */
+public class BlockInputStreamFactoryImpl implements BlockInputStreamFactory {
+
+  public static BlockInputStreamFactory getInstance() {
+    return new BlockInputStreamFactoryImpl();
+  }
+
+  /**
+   * Create a new BlockInputStream from the given parameters.
+   * @param blockId The blockID
+   * @param blockLen The Block Length
+   * @param pipeline The pipeline used to read from the block
+   * @param token The block access token
+   * @param verifyChecksum Whether to verify checksums or not
+   * @param xceiverFactory Factor to create the xceiver in the client
+   * @param refreshFunction Function to refresh the pipeline if needed.
+   * @return A BlockInputStream instance
+   */

Review comment:
       do we need this API?
   Can't we have one below API and make this inline there?

##########
File path: 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.ECBlockInputStream;
+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 =
+        new RatisReplicationConfig(HddsProtos.ReplicationFactor.THREE);
+
+    OmKeyLocationInfo blockInfo = createKeyLocationInfo(repConfig, 3,
+        1024*1024*10);

Review comment:
       NIt: format 1024*1024*10 -> 1024 * 1024 * 10

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
##########
@@ -19,60 +19,77 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.NotImplementedException;
-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.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.BlockInputStream;
-import org.apache.hadoop.hdds.scm.storage.ByteArrayReader;
-import org.apache.hadoop.hdds.scm.storage.ByteBufferReader;
+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.io.InputStream;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.function.Function;
 
 /**
  * Class to read data from an EC Block Group.
  */
-public class ECBlockInputStream extends InputStream
-    implements Seekable, CanUnbuffer, ByteBufferReadable {
+public class ECBlockInputStream extends BlockExtendedInputStream {
 
-  private static final int EOF = -1;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockInputStream.class);
 
   private final ECReplicationConfig repConfig;
+  // TODO - HDDS-5741 - remove hardcoded value

Review comment:
       HOw about to keep hardcode value in ECREplicationCOnfig java object? SO that we don't need to change all of this once we started to load the ecChunkSize as part of ECReplicationConfig?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
##########
@@ -19,60 +19,77 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.NotImplementedException;
-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.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.BlockInputStream;
-import org.apache.hadoop.hdds.scm.storage.ByteArrayReader;
-import org.apache.hadoop.hdds.scm.storage.ByteBufferReader;
+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.io.InputStream;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.function.Function;
 
 /**
  * Class to read data from an EC Block Group.
  */
-public class ECBlockInputStream extends InputStream
-    implements Seekable, CanUnbuffer, ByteBufferReadable {
+public class ECBlockInputStream extends BlockExtendedInputStream {
 
-  private static final int EOF = -1;
+  private static final Logger LOG =
+      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 BlockInputStreamProvider streamProvider;
+  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 DatanodeDetails[] parityLocations;
-  private final BlockInputStream[] blockStreams;
+  private final BlockExtendedInputStream[] blockStreams;
   private final int maxLocations;
 
-  private int position = 0;
+  private long position = 0;
   private boolean closed = false;
 
+  public ECBlockInputStream(ECReplicationConfig repConfig,
+      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,
-      BlockInputStreamProvider streamProvider) {
+      XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
     this.repConfig = repConfig;
+    // TODO - HDDS-5741
     this.ecChunkSize = ecChunkSize;
     this.verifyChecksum = verifyChecksum;
     this.blockInfo = blockInfo;
-    this.streamProvider = streamProvider;
+    this.streamFactory = streamFactory;
+    this.xceiverClientFactory = xceiverClientFactory;
+    this.refreshFunction = refreshFunction;
     this.maxLocations = repConfig.getData() + repConfig.getParity();
     this.dataLocations = new DatanodeDetails[repConfig.getData()];
     this.parityLocations = new DatanodeDetails[repConfig.getParity()];
-    this.blockStreams = new BlockInputStream[repConfig.getData()];
+    this.blockStreams = new BlockExtendedInputStream[repConfig.getData()];

Review comment:
       SO the plan is to lazily initialize parity stream when failures right? 
   Could you you give me high level idea on failure tracking?

##########
File path: hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneECClient.java
##########
@@ -184,29 +184,12 @@ public void testPutECKeyAndReadContent() throws IOException {
     Assert.assertEquals(keyName, key.getName());
     try (OzoneInputStream is = bucket.readKey(keyName)) {
       byte[] fileContent = new byte[1024];
-      Assert.assertEquals(inputChunks[0].length, is.read(fileContent));
-      Assert.assertEquals(new String(inputChunks[0], UTF_8),
-          new String(fileContent, UTF_8));
-    }
-
-    // Since EC read is not ready yet, let's use the regular read by
-    // tweaking the pipeline.
-    // Remove first node in EC pipeline. So, regular read will hit the
-    // first node in pipeline and assert for second chunk in EC data.
-    updatePipelineToKeepSingleNode(2);
-    try (OzoneInputStream is = bucket.readKey(keyName)) {
-      byte[] fileContent = new byte[1024];
-      Assert.assertEquals(inputChunks[1].length, is.read(fileContent));
-      Assert.assertEquals(new String(inputChunks[1], UTF_8),
-          new String(fileContent, UTF_8));
-    }
-
-    updatePipelineToKeepSingleNode(3);
-    try (OzoneInputStream is = bucket.readKey(keyName)) {
-      byte[] fileContent = new byte[1024];
-      Assert.assertEquals(inputChunks[2].length, is.read(fileContent));
-      Assert.assertEquals(new String(inputChunks[2], UTF_8),
-          new String(fileContent, UTF_8));
+      for (int i=0; i<3; i++) {

Review comment:
       Nit: You may want to use dataBlocks variable in this class.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] umamaheswararao commented on pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#issuecomment-924615457


   Latest changes looks good to me. I think if we wanted to moved hard coded value to ECReplicationCOnfig, please go ahead, we can get that done and refine delta. Other than that everything looks good to me. Thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] umamaheswararao commented on pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#issuecomment-925010222


   I don't want to make it bigger and confuse
   Just to be clarify: My proposed suggestion was not to make big. Its just matter to move the hard coded value from InputStream classes to ECReplConfig and use that. But it's not worth to wait this much for that change and blocked on that small change. :-) Let's move on to other JIRAs. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] umamaheswararao edited a comment on pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
umamaheswararao edited a comment on pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#issuecomment-925010222


   > I don't want to make it bigger and confuse
   Just to be clarify: My proposed suggestion was not to make big. Its just matter to move the hard coded value from InputStream classes to ECReplConfig and use that. But it's not worth to wait this much for that change and blocked on that small change. :-) Let's move on to other JIRAs. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] sodonnel commented on a change in pull request #2653: HDDS-5550. EC: Adapt KeyInputStream to read EC Blocks

Posted by GitBox <gi...@apache.org>.
sodonnel commented on a change in pull request #2653:
URL: https://github.com/apache/ozone/pull/2653#discussion_r712469716



##########
File path: 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.ECBlockInputStream;
+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 =
+        new RatisReplicationConfig(HddsProtos.ReplicationFactor.THREE);
+
+    OmKeyLocationInfo blockInfo = createKeyLocationInfo(repConfig, 3,
+        1024*1024*10);

Review comment:
       Fixed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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