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/08/17 15:54:48 UTC

[GitHub] [ozone] umamaheswararao commented on a change in pull request #2507: HDDS-4942. EC: Implement ECBlockInputStream to read a single EC BlockGroup

umamaheswararao commented on a change in pull request #2507:
URL: https://github.com/apache/ozone/pull/2507#discussion_r690464004



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
##########
@@ -0,0 +1,322 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.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.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.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.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Class to read data from an EC Block Group.
+ */
+public class ECBlockInputStream extends InputStream

Review comment:
       Is it make more sense to name it as BlockGroup InputStream?

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java
##########
@@ -0,0 +1,405 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+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.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+/**
+ * Tests for ECBlockInputStream.
+ */
+public class TestECBlockInputStream {
+
+  private static final int ONEMB = 1024 * 1024;
+
+  @Test
+  // TODO - this test will need changed when we can do recovery reads.
+  public void testSufficientLocations() {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+
+    // EC-3-2, 5MB block, so all 3 data locations are needed
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 5 * ONEMB);
+    ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, very large block, so all 3 data locations are needed
+    keyInfo = createKeyInfo(repConfig, 5, 5000 * ONEMB);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+    // EC-3-2, 1 byte short of 1MB with 1 location
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = createKeyInfo(repConfig, ONEMB - 1, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 1MB block but only location is in slot 2 (should never happen)
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 2);
+    keyInfo = createKeyInfo(repConfig, ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 5MB blocks, only 2 locations passed so we do not have sufficient
+    // locations.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 5MB blocks, only 1 data and 2 parity locations present. For now
+    // this will fail as we don't support reconstruction reads yet.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 4);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 5);
+    keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStream() throws IOException {
+    ByteBuffer buf = ByteBuffer.allocate(3*ONEMB);

Review comment:
       Nit: Format missing.

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java
##########
@@ -0,0 +1,405 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+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.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+/**
+ * Tests for ECBlockInputStream.
+ */
+public class TestECBlockInputStream {
+
+  private static final int ONEMB = 1024 * 1024;
+
+  @Test
+  // TODO - this test will need changed when we can do recovery reads.
+  public void testSufficientLocations() {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+
+    // EC-3-2, 5MB block, so all 3 data locations are needed
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 5 * ONEMB);
+    ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, very large block, so all 3 data locations are needed
+    keyInfo = createKeyInfo(repConfig, 5, 5000 * ONEMB);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+    // EC-3-2, 1 byte short of 1MB with 1 location
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = createKeyInfo(repConfig, ONEMB - 1, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 1MB block but only location is in slot 2 (should never happen)
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 2);
+    keyInfo = createKeyInfo(repConfig, ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 5MB blocks, only 2 locations passed so we do not have sufficient
+    // locations.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 5MB blocks, only 1 data and 2 parity locations present. For now
+    // this will fail as we don't support reconstruction reads yet.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 4);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 5);
+    keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStream() throws IOException {
+    ByteBuffer buf = ByteBuffer.allocate(3*ONEMB);
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+
+    TestBlockInputStreamFactory streamFactory =
+        new TestBlockInputStreamFactory();
+    ECBlockInputStream ecb = null;
+    List<TestBlockInputStream> streams = null;
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, ONEMB - 100);
+
+    // Block less than a single cell
+    try {
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      ecb.read(buf);
+      // We expect only 1 block stream and it should have a length passed of
+      // ONEMB - 100.
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB - 100, streams.get(0).getLength());
+    } finally {
+      if (ecb != null) {
+        ecb.close();
+      }
+    }
+
+    // Block of two cells
+    try {
+      streamFactory = new TestBlockInputStreamFactory();
+      keyInfo = createKeyInfo(repConfig, 5, ONEMB + 100);
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      buf.clear();
+      ecb.read(buf);
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(100, streams.get(1).getLength());
+    } finally {
+      ecb.close();
+    }
+
+    // Block of three cells
+    try {
+      streamFactory = new TestBlockInputStreamFactory();
+      keyInfo = createKeyInfo(repConfig, 5, 2 * ONEMB + 100);
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      buf.clear();
+      ecb.read(buf);
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(ONEMB, streams.get(1).getLength());
+      Assert.assertEquals(100, streams.get(2).getLength());
+    } finally {
+      ecb.close();
+    }
+
+    // Block of 3 full stripes and a partial stripe
+    try {
+      streamFactory = new TestBlockInputStreamFactory();
+      keyInfo = createKeyInfo(repConfig, 5, 10 * ONEMB + 100);
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      buf.clear();
+      ecb.read(buf);
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(4 * ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(3 * ONEMB + 100, streams.get(1).getLength());
+      Assert.assertEquals(3 * ONEMB, streams.get(2).getLength());
+    } finally {
+      ecb.close();
+    }
+
+    // Single Full cell
+    try {
+      streamFactory = new TestBlockInputStreamFactory();
+      keyInfo = createKeyInfo(repConfig, 5, ONEMB);
+      ecb = new ECBlockInputStream(repConfig, ONEMB,

Review comment:
       you may want to put in try stream, so it will be closed automatically

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java
##########
@@ -0,0 +1,405 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+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.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+/**
+ * Tests for ECBlockInputStream.
+ */
+public class TestECBlockInputStream {
+
+  private static final int ONEMB = 1024 * 1024;
+
+  @Test
+  // TODO - this test will need changed when we can do recovery reads.
+  public void testSufficientLocations() {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+
+    // EC-3-2, 5MB block, so all 3 data locations are needed
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 5 * ONEMB);
+    ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, very large block, so all 3 data locations are needed
+    keyInfo = createKeyInfo(repConfig, 5, 5000 * ONEMB);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+    // EC-3-2, 1 byte short of 1MB with 1 location
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = createKeyInfo(repConfig, ONEMB - 1, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 1MB block but only location is in slot 2 (should never happen)
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 2);
+    keyInfo = createKeyInfo(repConfig, ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 5MB blocks, only 2 locations passed so we do not have sufficient
+    // locations.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 5MB blocks, only 1 data and 2 parity locations present. For now
+    // this will fail as we don't support reconstruction reads yet.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 4);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 5);
+    keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStream() throws IOException {

Review comment:
       Should we split them into separate tests?

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java
##########
@@ -0,0 +1,405 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+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.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+/**
+ * Tests for ECBlockInputStream.
+ */
+public class TestECBlockInputStream {
+
+  private static final int ONEMB = 1024 * 1024;
+
+  @Test
+  // TODO - this test will need changed when we can do recovery reads.
+  public void testSufficientLocations() {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+
+    // EC-3-2, 5MB block, so all 3 data locations are needed
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 5 * ONEMB);
+    ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, very large block, so all 3 data locations are needed
+    keyInfo = createKeyInfo(repConfig, 5, 5000 * ONEMB);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+    // EC-3-2, 1 byte short of 1MB with 1 location
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = createKeyInfo(repConfig, ONEMB - 1, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertTrue(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 1MB block but only location is in slot 2 (should never happen)
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 2);
+    keyInfo = createKeyInfo(repConfig, ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 5MB blocks, only 2 locations passed so we do not have sufficient
+    // locations.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+
+    // EC-3-2, 5MB blocks, only 1 data and 2 parity locations present. For now
+    // this will fail as we don't support reconstruction reads yet.
+    dnMap.clear();
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 4);
+    dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 5);
+    keyInfo = createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+    ecb = new ECBlockInputStream(repConfig, ONEMB,
+        keyInfo, true, new TestBlockInputStreamFactory());
+    Assert.assertFalse(ecb.hasSufficientLocations());
+    ecb.close();
+  }
+
+  @Test
+  public void testCorrectBlockSizePassedToBlockStream() throws IOException {
+    ByteBuffer buf = ByteBuffer.allocate(3*ONEMB);
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+
+    TestBlockInputStreamFactory streamFactory =
+        new TestBlockInputStreamFactory();
+    ECBlockInputStream ecb = null;
+    List<TestBlockInputStream> streams = null;
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, ONEMB - 100);
+
+    // Block less than a single cell
+    try {
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      ecb.read(buf);
+      // We expect only 1 block stream and it should have a length passed of
+      // ONEMB - 100.
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB - 100, streams.get(0).getLength());
+    } finally {
+      if (ecb != null) {
+        ecb.close();
+      }
+    }
+
+    // Block of two cells
+    try {
+      streamFactory = new TestBlockInputStreamFactory();
+      keyInfo = createKeyInfo(repConfig, 5, ONEMB + 100);
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      buf.clear();
+      ecb.read(buf);
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(100, streams.get(1).getLength());
+    } finally {
+      ecb.close();
+    }
+
+    // Block of three cells
+    try {
+      streamFactory = new TestBlockInputStreamFactory();
+      keyInfo = createKeyInfo(repConfig, 5, 2 * ONEMB + 100);
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      buf.clear();
+      ecb.read(buf);
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(ONEMB, streams.get(1).getLength());
+      Assert.assertEquals(100, streams.get(2).getLength());
+    } finally {
+      ecb.close();
+    }
+
+    // Block of 3 full stripes and a partial stripe
+    try {
+      streamFactory = new TestBlockInputStreamFactory();
+      keyInfo = createKeyInfo(repConfig, 5, 10 * ONEMB + 100);
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      buf.clear();
+      ecb.read(buf);
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(4 * ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(3 * ONEMB + 100, streams.get(1).getLength());
+      Assert.assertEquals(3 * ONEMB, streams.get(2).getLength());
+    } finally {
+      ecb.close();
+    }
+
+    // Single Full cell
+    try {
+      streamFactory = new TestBlockInputStreamFactory();
+      keyInfo = createKeyInfo(repConfig, 5, ONEMB);
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      buf.clear();
+      ecb.read(buf);
+      // We expect 3 block streams and it should have a length passed of
+      // ONEMB and 100.
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(ONEMB, streams.get(0).getLength());
+      ecb.close();
+    } finally {
+      ecb.close();
+    }
+
+    // Several full stripes
+    try {
+      streamFactory = new TestBlockInputStreamFactory();
+      keyInfo = createKeyInfo(repConfig, 5, 9 * ONEMB);
+      ecb = new ECBlockInputStream(repConfig, ONEMB,
+          keyInfo, true, streamFactory);
+      buf.clear();
+      ecb.read(buf);
+      // We expect 3 block streams and it should have a length passed of
+      // ONEMB and 100.
+      streams = streamFactory.getBlockStreams();
+      Assert.assertEquals(3 * ONEMB, streams.get(0).getLength());
+      Assert.assertEquals(3 * ONEMB, streams.get(1).getLength());
+      Assert.assertEquals(3 * ONEMB, streams.get(2).getLength());
+    } finally {
+      ecb.close();
+    }
+  }
+
+  @Test
+  public void testSimpleRead() throws IOException {
+    ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+    TestBlockInputStreamFactory streamFactory =
+        new TestBlockInputStreamFactory();
+
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 5, 5 * ONEMB);
+    ECBlockInputStream ecb = new ECBlockInputStream(repConfig, ONEMB,

Review comment:
       same a above. Use try for stream

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
##########
@@ -0,0 +1,322 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.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.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.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.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Class to read data from an EC Block Group.
+ */
+public class ECBlockInputStream extends InputStream
+    implements Seekable, CanUnbuffer, ByteBufferReadable {
+
+  private static final int EOF = -1;
+
+  private final ECReplicationConfig repConfig;
+  private final int ecChunkSize;
+  private final BlockInputStreamFactory streamFactory;
+  private final boolean verifyChecksum;
+  private final OmKeyLocationInfo blockInfo;
+  private final DatanodeDetails[] dataLocations;
+  private final DatanodeDetails[] parityLocations;
+  private final BlockInputStream[] blockStreams;
+  private final int maxLocations;
+
+  private int position = 0;
+  private boolean closed = false;
+
+  public ECBlockInputStream(ECReplicationConfig repConfig, int ecChunkSize,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+      BlockInputStreamFactory streamFactory) {
+    this.repConfig = repConfig;
+    this.ecChunkSize = ecChunkSize;
+    this.verifyChecksum = verifyChecksum;
+    this.blockInfo = blockInfo;
+    this.streamFactory = streamFactory;
+    this.maxLocations = repConfig.getData() + repConfig.getParity();
+    this.dataLocations = new DatanodeDetails[repConfig.getData()];
+    this.parityLocations = new DatanodeDetails[repConfig.getParity()];
+    this.blockStreams = new BlockInputStream[repConfig.getData()];
+
+    setBlockLocations(this.blockInfo.getPipeline());
+  }
+
+  public synchronized boolean hasSufficientLocations() {
+    // Until we implement "on the fly" recovery, all data location must be
+    // present and we have enough locations if that is the case.
+    //
+    // The number of locations needed is a function of the EC Chunk size. If the
+    // block length is <= the chunk size, we should only have location 1. If it
+    // is greater than the chunk size but less than chunk_size * 2, then we must
+    // have two locations. If it is greater than chunk_size * data_num, then we
+    // must have all data_num locations.
+    int expectedDataBlocks =
+        (int)Math.min(
+            Math.ceil((double)blockInfo.getLength() / ecChunkSize),
+            repConfig.getData());
+    for (int i=0; i<expectedDataBlocks; i++) {
+      if (dataLocations[i] == null) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Using the current position, returns the index of the blockStream we should
+   * be reading from. This is the index in the internal array holding the
+   * stream reference. The block group index will be one greater than this.
+   * @return
+   */
+  private int currentStreamIndex() {
+    return ((position / ecChunkSize) % repConfig.getData());
+  }
+
+  /**
+   * Uses the current position and ecChunkSize to determine which of the
+   * internal block streams the next read should come from. Also opens the
+   * stream if it has not been opened already.
+   * @return BlockInput stream to read from.
+   */
+  private BlockInputStream getOrOpenStream() {
+    int ind = currentStreamIndex();
+    BlockInputStream stream = blockStreams[ind];
+    if (stream == null) {
+      // To read an EC block, we create a STANDALONE pipeline that contains the
+      // single location for the block index we want to read. The EC blocks are
+      // indexed from 1 to N, however the data locations are stored in the
+      // dataLocations array indexed from zero.
+      Pipeline pipeline = Pipeline.newBuilder()
+          .setReplicationConfig(new StandaloneReplicationConfig(
+              HddsProtos.ReplicationFactor.ONE))
+          .setNodes(Arrays.asList(dataLocations[ind]))
+          .setId(PipelineID.randomId())

Review comment:
       So there is no pipeline Id validation at DN? example if the writer written the block with pipeline id 10, then reader can use any id?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+import java.util.function.Function;
+
+/**
+ * Concrete implementation of a BlockInputStreamFactory to create
+ * BlockInputSteams in a real cluster.
+ */
+public class BlockInputStreamFactoryImpl implements BlockInputStreamFactory {
+
+  private final XceiverClientFactory xceiverClientFactory;
+  private final Function<BlockID, Pipeline> refreshFunction;
+
+  public BlockInputStreamFactoryImpl(XceiverClientFactory xceiverFactory,
+      Function<BlockID, Pipeline> refreshFunction) {
+    this.xceiverClientFactory = xceiverFactory;
+    this.refreshFunction = refreshFunction;
+  }
+
+  @Override
+  public BlockInputStream create(BlockID blockId, long blockLen,
+      Pipeline pipeline, Token<OzoneBlockTokenIdentifier> token,
+      boolean verifyChecksum) {
+    return new BlockInputStream(blockId, blockLen, pipeline, token,

Review comment:
       We can use factory when we need to create different category of objects of same type. Do you have plan to create different type of blockInputStream here? If yes, then what parameter decides the type?




-- 
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