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/11/03 11:11:06 UTC

[GitHub] [ozone] sodonnel opened a new pull request #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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


   ## What changes were proposed in this pull request?
   
   This PR introduces a class called ECBlockReconstructedStripeInputStream. This class is intended to read an EC block when one or more of the data blocks are missing. Its interface allows for full EC stripes of data to be returned to the client, where some of the data is reconstructed via EC Decoding.
   
   It is not intended for this class to be part of a public facing API. It will instead be used inside a public "EC Input Stream" class which will decide to use a non-reconstruction or reconstruction read depending on how many locations are available or how if any reads fail. This public class is yet to be adapted to use this new class, and that will be taken care of in further PR.
   
   As part of this change, changes were made to ECBlockInputStream to allow some common code to be shared by this new class, which extends ECBlockInputStream.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5551
   
   ## How was this patch tested?
   
   New unit 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] umamaheswararao commented on a change in pull request #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,419 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  };
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }
+    Collections.shuffle(indexes);
+    List<Integer> picked = indexes.subList(0, numRequired);
+    Collections.sort(picked);
+    return picked;
+  }
+
+  private ByteBuffer allocateBuffer(ECReplicationConfig repConfig) {
+    ByteBuffer buf = ByteBuffer.allocate(repConfig.getEcChunkSize());
+    return buf;
+  }
+
+  private void flipInputs() {
+    for (ByteBuffer b : decoderInputBuffers) {
+      if (b != null) {
+        b.flip();
+      }
+    }
+  }
+
+  private void clearParityBuffers() {
+    for (int i = getRepConfig().getData();
+         i < getRepConfig().getRequiredNodes(); i++) {
+      if (decoderInputBuffers[i] != null) {
+        decoderInputBuffers[i].clear();
+      }
+    }
+  }
+
+  protected void loadDataBuffersFromStream() throws IOException {
+    for (int i = 0; i < dataIndexes.size(); i++) {
+      BlockExtendedInputStream stream =
+          getOrOpenStream(i, dataIndexes.get(i));
+      ByteBuffer b = decoderInputBuffers[dataIndexes.get(i)];
+      while (b.hasRemaining()) {
+        int read = stream.read(b);
+        if (read == EOF) {
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Take the populated input buffers and missing indexes and create the
+   * outputs. Note that the input buffers have to be "ready for read", ie they
+   * need to have been flipped after their data was loaded. The created outputs
+   * are "ready to read" by the underlying decoder API, so there is no need to
+   * flip them after the call. The decoder reads all the inputs leaving the
+   * buffer position at the end, so the inputs are flipped after the decode so
+   * we have a complete set of "outputs" for the EC Stripe which are ready to
+   * read.
+   * @throws IOException
+   */
+  private void decodeStripe() throws IOException {
+    decoder.decode(decoderInputBuffers, missingIndexes, decoderOutputBuffers);
+    flipInputs();
+  }
+
+  @Override
+  public synchronized boolean hasSufficientLocations() {
+    // The number of locations needed is a function of the EC Chunk size. If the
+    // block length is <= the chunk size, we should only have one data location.
+    // is greater than the chunk size but less than chunk_size * 2, then we must

Review comment:
       is greater than --> If it is greater than




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();

Review comment:
       loadDataBuffersFromStream will have some better error handling when seek is implemented (HDDS-5950) to ensure enough bytes are read.
   
   HDDS-5951 is for handing failures as the reader is progressing.
   
   We could issue reads in parallel here as a further improvement - HDDS-5952




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {

Review comment:
       Should we use pool for allocating buffers? Which can be reused across block-groups

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */
+public class TestECBlockReconstructedStripeInputStream {
+
+
+  private static final int ONEMB = 1024 * 1024;
+
+  private ECReplicationConfig repConfig;
+  private TestBlockInputStreamFactory streamFactory;
+
+  @Before
+  public void setup() {
+    repConfig = new ECReplicationConfig(3, 2,
+        ECReplicationConfig.EcCodec.RS, ONEMB);
+    streamFactory = new TestBlockInputStreamFactory();
+  }
+
+  @Test
+  public void testSufficientLocations() {
+    // One chunk, only 1 location.
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 1, ONEMB);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+    // Two Chunks, but missing data block 2.
+    dnMap = createIndexMap(1, 4, 5);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 2, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3.
+    dnMap = createIndexMap(1, 4, 5);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3 and parity 1.
+    dnMap = createIndexMap(1, 4);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+  }
+
+  @Test
+  public void testReadFullStripesWithPartial() throws IOException {
+    // Generate the input data for 3 full stripes and generate the parity.
+    int chunkSize = repConfig.getEcChunkSize();
+    int partialStripeSize = chunkSize * 2 - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+    dataBufs[1].limit(4 * chunkSize - 1);
+    dataBufs[2].limit(3 * chunkSize);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(createIndexMap(1, 4, 5));
+    // One data missing
+    locations.add(createIndexMap(1, 2, 4, 5));
+    // Two data missing including first
+    locations.add(createIndexMap(2, 4, 5));
+    // One data and one parity missing
+    locations.add(createIndexMap(2, 3, 4));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+
+      OmKeyLocationInfo keyInfo = createKeyInfo(repConfig,
+          stripeSize() * 3 + partialStripeSize, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+      try (ECBlockReconstructedStripeInputStream ecb =
+          new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+              null, null, streamFactory)) {
+        // Read 3 full stripes
+        for (int i = 0; i < 3; i++) {
+          int read = ecb.readStripe(bufs);
+          for (int j = 0; j < bufs.length; j++) {
+            validateContents(dataBufs[j], bufs[j], i * chunkSize, chunkSize);
+          }
+          Assert.assertEquals(stripeSize(), read);
+
+          // Check the underlying streams have read 1 chunk per read:
+          for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+            Assert.assertEquals(chunkSize * (i + 1),
+                bis.getPos());
+          }
+          Assert.assertEquals(stripeSize() * (i + 1), ecb.getPos());
+          clearBuffers(bufs);
+        }
+        // The next read is a partial stripe
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(partialStripeSize, read);
+        validateContents(dataBufs[0], bufs[0], 3 * chunkSize, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 3 * chunkSize, chunkSize - 1);
+        Assert.assertEquals(0, bufs[2].remaining());
+        Assert.assertEquals(0, bufs[2].position());
+
+        // A further read should give EOF
+        clearBuffers(bufs);
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  @Test
+  public void testReadPartialStripe() throws IOException {
+    int blockLength = repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(blockLength);
+    dataBufs[1].limit(0);
+    dataBufs[2].limit(0);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap = createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    try (ECBlockReconstructedStripeInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+            null, null, streamFactory)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      validateContents(dataBufs[0], bufs[0], 0, blockLength);
+      Assert.assertEquals(0, bufs[1].remaining());
+      Assert.assertEquals(0, bufs[1].position());
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 blockLength:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(blockLength, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeTwoChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 2 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(chunkSize);
+    dataBufs[1].limit(chunkSize - 1);
+    dataBufs[2].limit(0);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap = createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    try (ECBlockReconstructedStripeInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+            null, null, streamFactory)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      validateContents(dataBufs[0], bufs[0], 0, chunkSize);
+      validateContents(dataBufs[1], bufs[1], 0, chunkSize - 1);
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 chunk:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(chunkSize, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeThreeChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 3 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(chunkSize);
+    dataBufs[1].limit(chunkSize);
+    dataBufs[2].limit(chunkSize - 1);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    // We have a length that is less than a stripe, so chunks 1 and 2 are full.
+    // Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(createIndexMap(3, 4, 5));
+    // Two data missing
+    locations.add(createIndexMap(1, 4, 5));
+    // One data missing - the last one
+    locations.add(createIndexMap(1, 2, 5));
+    // One data and one parity missing
+    locations.add(createIndexMap(2, 3, 4));
+    // One data and one parity missing
+    locations.add(createIndexMap(1, 2, 4));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+
+      OmKeyLocationInfo keyInfo =
+          createKeyInfo(repConfig, blockLength, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+      try (ECBlockReconstructedStripeInputStream ecb =
+          new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+              null, null, streamFactory)) {
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(blockLength, read);
+        validateContents(dataBufs[0], bufs[0], 0, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 0, chunkSize);
+        validateContents(dataBufs[2], bufs[2], 0, chunkSize - 1);
+        // Check the underlying streams have been advanced by 1 chunk:
+        for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+          Assert.assertEquals(0, bis.getRemaining());
+        }
+        Assert.assertEquals(ecb.getPos(), blockLength);
+        clearBuffers(bufs);
+        // A further read should give EOF
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  private void addDataStreamsToFactory(ByteBuffer[] data, ByteBuffer[] parity) {
+    List<ByteBuffer> dataStreams = new ArrayList<>();
+    for (ByteBuffer b : data) {
+      dataStreams.add(b);
+    }
+    for (ByteBuffer b : parity) {
+      dataStreams.add(b);
+    }
+    streamFactory.setBlockStreamData(dataStreams);
+  }
+
+  /**
+   * Validates that the data buffer has the same contents as the source buffer,
+   * starting the checks in the src at offset and for count bytes.
+   * @param src The source of the data
+   * @param data The data which should be checked against the source
+   * @param offset The starting point in the src buffer
+   * @param count How many bytes to check.
+   */
+  private void validateContents(ByteBuffer src, ByteBuffer data, int offset,
+      int count) {
+    byte[] srcArray = src.array();
+    Assert.assertEquals(count, data.remaining());
+    for (int i = offset; i < offset + count; i++) {
+      Assert.assertEquals("Element " + i, srcArray[i], data.get());
+    }
+    data.flip();
+  }
+
+  /**
+   * Returns a new map containing a random DatanodeDetails for each index in
+   * inputs.
+   * @param idxs A list of indexes to add to the map
+   * @return A map of DatanodeDetails to index.
+   */
+  private Map<DatanodeDetails, Integer> createIndexMap(int... idxs) {
+    Map<DatanodeDetails, Integer> map = new HashMap<>();
+    for (int i : idxs) {
+      map.put(MockDatanodeDetails.randomDatanodeDetails(), i);
+    }
+    return map;
+  }
+
+  /**
+   * Given a set of data buffers, generate the parity data for the inputs.
+   * @param data A set of data buffers
+   * @param ecConfig The ECReplicationConfig representing the scheme
+   * @return
+   * @throws IOException
+   */
+  private ByteBuffer[] generateParity(ByteBuffer[] data,
+      ECReplicationConfig ecConfig) throws IOException {
+    // First data buffer dictates the size
+    int cellSize = data[0].limit();
+    // Store the positions of the remaining data buffers so we can restore them
+    int[] dataLimits = new int[data.length];
+    for (int i=1; i<data.length; i++) {
+      dataLimits[i] = data[i].limit();
+      data[i].limit(cellSize);
+      zeroFill(data[i]);
+      data[i].flip();
+    }
+    ByteBuffer[] parity = new ByteBuffer[ecConfig.getParity()];
+    for (int i=0; i<ecConfig.getParity(); i++) {

Review comment:
       Nit: There are some places code formatting off. Exampleabove line. 




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  }
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }
+    Preconditions.assertTrue(indexes.size() >= numRequired);
+    Random rand = new Random();
+    while (indexes.size() > numRequired) {
+      indexes.remove(rand.nextInt(indexes.size()));
+    }
+    return indexes;
+  }
+
+  private ByteBuffer allocateBuffer(ECReplicationConfig repConfig) {
+    ByteBuffer buf = ByteBuffer.allocate(repConfig.getEcChunkSize());
+    return buf;
+  }
+
+  private void flipInputs() {
+    for (ByteBuffer b : decoderInputBuffers) {
+      if (b != null) {
+        b.flip();
+      }
+    }
+  }
+
+  private void clearParityBuffers() {
+    for (int i = getRepConfig().getData();
+         i < getRepConfig().getRequiredNodes(); i++) {
+      if (decoderInputBuffers[i] != null) {
+        decoderInputBuffers[i].clear();
+      }
+    }
+  }
+
+  protected void loadDataBuffersFromStream() throws IOException {
+    for (int i = 0; i < dataIndexes.size(); i++) {
+      BlockExtendedInputStream stream =
+          getOrOpenStream(i, dataIndexes.get(i));
+      ByteBuffer b = decoderInputBuffers[dataIndexes.get(i)];
+      while (b.hasRemaining()) {
+        int read = stream.read(b);
+        if (read == EOF) {
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Take the populated input buffers and missing indexes and create the
+   * outputs. Note that the input buffers have to be "ready for read", ie they
+   * need to have been flipped after their data was loaded. The created outputs
+   * are "ready to read" by the underlying decoder API, so there is no need to
+   * flip them after the call. The decoder reads all the inputs leaving the
+   * buffer position at the end, so the inputs are flipped after the decode so
+   * we have a complete set of "outputs" for the EC Stripe which are ready to
+   * read.
+   * @throws IOException
+   */
+  private void decodeStripe() throws IOException {
+    decoder.decode(decoderInputBuffers, missingIndexes, decoderOutputBuffers);
+    flipInputs();
+  }
+
+  @Override

Review comment:
       isn't it this method logic can moved into super class method?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  }
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }
+    Preconditions.assertTrue(indexes.size() >= numRequired);
+    Random rand = new Random();
+    while (indexes.size() > numRequired) {
+      indexes.remove(rand.nextInt(indexes.size()));
+    }
+    return indexes;
+  }
+
+  private ByteBuffer allocateBuffer(ECReplicationConfig repConfig) {
+    ByteBuffer buf = ByteBuffer.allocate(repConfig.getEcChunkSize());
+    return buf;
+  }
+
+  private void flipInputs() {
+    for (ByteBuffer b : decoderInputBuffers) {
+      if (b != null) {
+        b.flip();
+      }
+    }
+  }
+
+  private void clearParityBuffers() {
+    for (int i = getRepConfig().getData();
+         i < getRepConfig().getRequiredNodes(); i++) {
+      if (decoderInputBuffers[i] != null) {
+        decoderInputBuffers[i].clear();
+      }
+    }
+  }
+
+  protected void loadDataBuffersFromStream() throws IOException {
+    for (int i = 0; i < dataIndexes.size(); i++) {
+      BlockExtendedInputStream stream =
+          getOrOpenStream(i, dataIndexes.get(i));
+      ByteBuffer b = decoderInputBuffers[dataIndexes.get(i)];
+      while (b.hasRemaining()) {
+        int read = stream.read(b);
+        if (read == EOF) {
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Take the populated input buffers and missing indexes and create the
+   * outputs. Note that the input buffers have to be "ready for read", ie they
+   * need to have been flipped after their data was loaded. The created outputs
+   * are "ready to read" by the underlying decoder API, so there is no need to
+   * flip them after the call. The decoder reads all the inputs leaving the
+   * buffer position at the end, so the inputs are flipped after the decode so
+   * we have a complete set of "outputs" for the EC Stripe which are ready to
+   * read.
+   * @throws IOException
+   */
+  private void decodeStripe() throws IOException {
+    decoder.decode(decoderInputBuffers, missingIndexes, decoderOutputBuffers);
+    flipInputs();
+  }
+
+  @Override
+  public synchronized boolean hasSufficientLocations() {
+    // The number of locations needed is a function of the EC Chunk size. If the
+    // block length is <= the chunk size, we should only have one data location.
+    // If it is greater than the chunk size but less than chunk_size * 2, then
+    // we must have two locations. If it is greater than chunk_size * data_num,
+    // then we must have all data_num locations.
+    // The remaining data locations (for small block lengths) can be assumed to
+    // be all zeros.
+    // Then we need a total of dataNum blocks available across the available
+    // data, parity and padding blocks.
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    int availableLocations =
+        availableDataLocations() + availableParityLocations();
+    int paddedLocations = repConfig.getData() - expectedDataBlocks;
+
+    if (availableLocations + paddedLocations >= repConfig.getData()) {
+      return true;
+    } else {
+      LOG.warn("There are insufficient locations. {} available {} padded {} " +
+          "expected", availableLocations, paddedLocations, expectedDataBlocks);
+      return false;
+    }
+  }
+

Review comment:
       Below method can moved to super class and use?
   Same calculation is present in ECBlockInputStream#hasSufficientLocations




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();

Review comment:
       loadDataBuffersFromStream: This method should be designed to return failures from streams right?
   I am assuming currently this patch does not handle the case: when OM retuns no locations for some of the replicas and we have to use this ECBlkReconstructedInputStream class. if another node down when calling ECBlkReconstructedInputStream#loadDataBuffersFromStream, it will simply fail. I remeber we discussed to cover that cases in other JIRAs by adjusting available locations and retrying. But wanted to make sure about the scope here. 
   Another improvement could be issueing reads parallelly here. But I am ok in this JIRA for now, but lets track the improvements.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,419 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  };
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }

Review comment:
       for (int i = 0; i < repConfig.getParity() - numRequired; i++) {
         indexes.remove(random.nextInt(indexes.size()));
   }
   return indexes;
   
   Just another thought: Something like this can achieve the below logic? Probably we should change from ArrayList to LinkedList to reduce random location remove cost.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
##########
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import java.io.IOException;
+
+/**
+ * Exception throw by EC Input Streams if there are not enough locations to

Review comment:
       thrown




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,419 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  };
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }

Review comment:
       I see what you mean - The number of elements is some small here (at most 4 for 10-4) and we only call this once per block - so a linked list may add more overhead than it saves us traversing the list to do the removals. I am not sure, but I have changed the code to be inline with your suggestion.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,419 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  };
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }

Review comment:
       Thanks for addressing it.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */

Review comment:
       I think there is one case missing is, partial stripe with first chunk has less than ecChunkSize? does this case covered? I might be missing please point it to me.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {

Review comment:
       I think we should create another Jira for this, and perhaps think about how we create a shared buffer pool that can be used for read and write across the life of the client. Raised HDDS-5949.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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


   Thanks a lot @sodonnel for addressing the comments. I have just looked again and found few more minors. Sorry for not posting in single time. Please check. 
   Once they are addressed/discussion, I think we are good to go.


-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  }
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }
+    Preconditions.assertTrue(indexes.size() >= numRequired);
+    Random rand = new Random();
+    while (indexes.size() > numRequired) {
+      indexes.remove(rand.nextInt(indexes.size()));
+    }
+    return indexes;
+  }
+
+  private ByteBuffer allocateBuffer(ECReplicationConfig repConfig) {
+    ByteBuffer buf = ByteBuffer.allocate(repConfig.getEcChunkSize());
+    return buf;
+  }
+
+  private void flipInputs() {
+    for (ByteBuffer b : decoderInputBuffers) {
+      if (b != null) {
+        b.flip();
+      }
+    }
+  }
+
+  private void clearParityBuffers() {
+    for (int i = getRepConfig().getData();
+         i < getRepConfig().getRequiredNodes(); i++) {
+      if (decoderInputBuffers[i] != null) {
+        decoderInputBuffers[i].clear();
+      }
+    }
+  }
+
+  protected void loadDataBuffersFromStream() throws IOException {
+    for (int i = 0; i < dataIndexes.size(); i++) {
+      BlockExtendedInputStream stream =
+          getOrOpenStream(i, dataIndexes.get(i));
+      ByteBuffer b = decoderInputBuffers[dataIndexes.get(i)];
+      while (b.hasRemaining()) {
+        int read = stream.read(b);
+        if (read == EOF) {
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Take the populated input buffers and missing indexes and create the
+   * outputs. Note that the input buffers have to be "ready for read", ie they
+   * need to have been flipped after their data was loaded. The created outputs
+   * are "ready to read" by the underlying decoder API, so there is no need to
+   * flip them after the call. The decoder reads all the inputs leaving the
+   * buffer position at the end, so the inputs are flipped after the decode so
+   * we have a complete set of "outputs" for the EC Stripe which are ready to
+   * read.
+   * @throws IOException
+   */
+  private void decodeStripe() throws IOException {
+    decoder.decode(decoderInputBuffers, missingIndexes, decoderOutputBuffers);
+    flipInputs();
+  }
+
+  @Override
+  public synchronized boolean hasSufficientLocations() {
+    // The number of locations needed is a function of the EC Chunk size. If the
+    // block length is <= the chunk size, we should only have one data location.
+    // If it is greater than the chunk size but less than chunk_size * 2, then
+    // we must have two locations. If it is greater than chunk_size * data_num,
+    // then we must have all data_num locations.
+    // The remaining data locations (for small block lengths) can be assumed to
+    // be all zeros.
+    // Then we need a total of dataNum blocks available across the available
+    // data, parity and padding blocks.
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    int availableLocations =
+        availableDataLocations() + availableParityLocations();
+    int paddedLocations = repConfig.getData() - expectedDataBlocks;
+
+    if (availableLocations + paddedLocations >= repConfig.getData()) {
+      return true;
+    } else {
+      LOG.warn("There are insufficient locations. {} available {} padded {} " +
+          "expected", availableLocations, paddedLocations, expectedDataBlocks);
+      return false;
+    }
+  }
+

Review comment:
       Yea, you are correct - this method can go into the super class and be re-used in this class and the super class. I've made that 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] sodonnel commented on a change in pull request #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */
+public class TestECBlockReconstructedStripeInputStream {
+
+
+  private static final int ONEMB = 1024 * 1024;
+
+  private ECReplicationConfig repConfig;
+  private TestBlockInputStreamFactory streamFactory;
+
+  @Before
+  public void setup() {
+    repConfig = new ECReplicationConfig(3, 2,
+        ECReplicationConfig.EcCodec.RS, ONEMB);
+    streamFactory = new TestBlockInputStreamFactory();
+  }
+
+  @Test
+  public void testSufficientLocations() {
+    // One chunk, only 1 location.
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 1, ONEMB);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+    // Two Chunks, but missing data block 2.
+    dnMap = createIndexMap(1, 4, 5);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 2, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3.
+    dnMap = createIndexMap(1, 4, 5);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3 and parity 1.
+    dnMap = createIndexMap(1, 4);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+  }
+
+  @Test
+  public void testReadFullStripesWithPartial() throws IOException {
+    // Generate the input data for 3 full stripes and generate the parity.
+    int chunkSize = repConfig.getEcChunkSize();
+    int partialStripeSize = chunkSize * 2 - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+    dataBufs[1].limit(4 * chunkSize - 1);
+    dataBufs[2].limit(3 * chunkSize);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(createIndexMap(1, 4, 5));
+    // One data missing
+    locations.add(createIndexMap(1, 2, 4, 5));
+    // Two data missing including first
+    locations.add(createIndexMap(2, 4, 5));
+    // One data and one parity missing
+    locations.add(createIndexMap(2, 3, 4));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+
+      OmKeyLocationInfo keyInfo = createKeyInfo(repConfig,
+          stripeSize() * 3 + partialStripeSize, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+      try (ECBlockReconstructedStripeInputStream ecb =
+          new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+              null, null, streamFactory)) {
+        // Read 3 full stripes
+        for (int i = 0; i < 3; i++) {
+          int read = ecb.readStripe(bufs);
+          for (int j = 0; j < bufs.length; j++) {
+            validateContents(dataBufs[j], bufs[j], i * chunkSize, chunkSize);
+          }
+          Assert.assertEquals(stripeSize(), read);
+
+          // Check the underlying streams have read 1 chunk per read:
+          for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+            Assert.assertEquals(chunkSize * (i + 1),
+                bis.getPos());
+          }
+          Assert.assertEquals(stripeSize() * (i + 1), ecb.getPos());
+          clearBuffers(bufs);
+        }
+        // The next read is a partial stripe
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(partialStripeSize, read);
+        validateContents(dataBufs[0], bufs[0], 3 * chunkSize, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 3 * chunkSize, chunkSize - 1);
+        Assert.assertEquals(0, bufs[2].remaining());
+        Assert.assertEquals(0, bufs[2].position());
+
+        // A further read should give EOF
+        clearBuffers(bufs);
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  @Test
+  public void testReadPartialStripe() throws IOException {
+    int blockLength = repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(blockLength);
+    dataBufs[1].limit(0);
+    dataBufs[2].limit(0);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap = createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    try (ECBlockReconstructedStripeInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+            null, null, streamFactory)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      validateContents(dataBufs[0], bufs[0], 0, blockLength);
+      Assert.assertEquals(0, bufs[1].remaining());
+      Assert.assertEquals(0, bufs[1].position());
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 blockLength:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(blockLength, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeTwoChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 2 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(chunkSize);
+    dataBufs[1].limit(chunkSize - 1);
+    dataBufs[2].limit(0);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap = createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    try (ECBlockReconstructedStripeInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+            null, null, streamFactory)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      validateContents(dataBufs[0], bufs[0], 0, chunkSize);
+      validateContents(dataBufs[1], bufs[1], 0, chunkSize - 1);
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 chunk:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(chunkSize, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeThreeChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 3 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(chunkSize);
+    dataBufs[1].limit(chunkSize);
+    dataBufs[2].limit(chunkSize - 1);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    // We have a length that is less than a stripe, so chunks 1 and 2 are full.
+    // Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(createIndexMap(3, 4, 5));
+    // Two data missing
+    locations.add(createIndexMap(1, 4, 5));
+    // One data missing - the last one
+    locations.add(createIndexMap(1, 2, 5));
+    // One data and one parity missing
+    locations.add(createIndexMap(2, 3, 4));
+    // One data and one parity missing
+    locations.add(createIndexMap(1, 2, 4));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+
+      OmKeyLocationInfo keyInfo =
+          createKeyInfo(repConfig, blockLength, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+      try (ECBlockReconstructedStripeInputStream ecb =
+          new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+              null, null, streamFactory)) {
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(blockLength, read);
+        validateContents(dataBufs[0], bufs[0], 0, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 0, chunkSize);
+        validateContents(dataBufs[2], bufs[2], 0, chunkSize - 1);
+        // Check the underlying streams have been advanced by 1 chunk:
+        for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+          Assert.assertEquals(0, bis.getRemaining());
+        }
+        Assert.assertEquals(ecb.getPos(), blockLength);
+        clearBuffers(bufs);
+        // A further read should give EOF
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  private void addDataStreamsToFactory(ByteBuffer[] data, ByteBuffer[] parity) {
+    List<ByteBuffer> dataStreams = new ArrayList<>();
+    for (ByteBuffer b : data) {
+      dataStreams.add(b);
+    }
+    for (ByteBuffer b : parity) {
+      dataStreams.add(b);
+    }
+    streamFactory.setBlockStreamData(dataStreams);
+  }
+
+  /**
+   * Validates that the data buffer has the same contents as the source buffer,
+   * starting the checks in the src at offset and for count bytes.
+   * @param src The source of the data
+   * @param data The data which should be checked against the source
+   * @param offset The starting point in the src buffer
+   * @param count How many bytes to check.
+   */
+  private void validateContents(ByteBuffer src, ByteBuffer data, int offset,
+      int count) {
+    byte[] srcArray = src.array();
+    Assert.assertEquals(count, data.remaining());
+    for (int i = offset; i < offset + count; i++) {
+      Assert.assertEquals("Element " + i, srcArray[i], data.get());
+    }
+    data.flip();
+  }
+
+  /**
+   * Returns a new map containing a random DatanodeDetails for each index in
+   * inputs.
+   * @param idxs A list of indexes to add to the map
+   * @return A map of DatanodeDetails to index.
+   */
+  private Map<DatanodeDetails, Integer> createIndexMap(int... idxs) {
+    Map<DatanodeDetails, Integer> map = new HashMap<>();
+    for (int i : idxs) {
+      map.put(MockDatanodeDetails.randomDatanodeDetails(), i);
+    }
+    return map;
+  }
+
+  /**
+   * Given a set of data buffers, generate the parity data for the inputs.
+   * @param data A set of data buffers
+   * @param ecConfig The ECReplicationConfig representing the scheme
+   * @return
+   * @throws IOException
+   */
+  private ByteBuffer[] generateParity(ByteBuffer[] data,
+      ECReplicationConfig ecConfig) throws IOException {
+    // First data buffer dictates the size
+    int cellSize = data[0].limit();
+    // Store the positions of the remaining data buffers so we can restore them
+    int[] dataLimits = new int[data.length];
+    for (int i=1; i<data.length; i++) {
+      dataLimits[i] = data[i].limit();
+      data[i].limit(cellSize);
+      zeroFill(data[i]);
+      data[i].flip();
+    }
+    ByteBuffer[] parity = new ByteBuffer[ecConfig.getParity()];
+    for (int i=0; i<ecConfig.getParity(); i++) {

Review comment:
       OK - I found a couple of instances and fixed them. TBH, I am not sure if `i=0` or `i = 0` is better. I personally prefer the first as I think its more readable in the for loops. However if we don't want to allow variable assignment without a space, then should we not update checkstyle to enforce it?




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  }
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }
+    Preconditions.assertTrue(indexes.size() >= numRequired);
+    Random rand = new Random();
+    while (indexes.size() > numRequired) {
+      indexes.remove(rand.nextInt(indexes.size()));
+    }
+    return indexes;
+  }
+
+  private ByteBuffer allocateBuffer(ECReplicationConfig repConfig) {
+    ByteBuffer buf = ByteBuffer.allocate(repConfig.getEcChunkSize());
+    return buf;
+  }
+
+  private void flipInputs() {
+    for (ByteBuffer b : decoderInputBuffers) {
+      if (b != null) {
+        b.flip();
+      }
+    }
+  }
+
+  private void clearParityBuffers() {
+    for (int i = getRepConfig().getData();
+         i < getRepConfig().getRequiredNodes(); i++) {
+      if (decoderInputBuffers[i] != null) {
+        decoderInputBuffers[i].clear();
+      }
+    }
+  }
+
+  protected void loadDataBuffersFromStream() throws IOException {
+    for (int i = 0; i < dataIndexes.size(); i++) {
+      BlockExtendedInputStream stream =
+          getOrOpenStream(i, dataIndexes.get(i));
+      ByteBuffer b = decoderInputBuffers[dataIndexes.get(i)];
+      while (b.hasRemaining()) {
+        int read = stream.read(b);
+        if (read == EOF) {
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Take the populated input buffers and missing indexes and create the
+   * outputs. Note that the input buffers have to be "ready for read", ie they
+   * need to have been flipped after their data was loaded. The created outputs
+   * are "ready to read" by the underlying decoder API, so there is no need to
+   * flip them after the call. The decoder reads all the inputs leaving the
+   * buffer position at the end, so the inputs are flipped after the decode so
+   * we have a complete set of "outputs" for the EC Stripe which are ready to
+   * read.
+   * @throws IOException
+   */
+  private void decodeStripe() throws IOException {
+    decoder.decode(decoderInputBuffers, missingIndexes, decoderOutputBuffers);
+    flipInputs();
+  }
+
+  @Override

Review comment:
       No - the calculation in the super class is different at the moment. The super class is for happy path reads where all data blocks are available, and it may be used to decide whether to use the ECBlockInputStream or ECBlockReconstructedStripeInputStream. I still need to think fully through how and where the client decides to use ECBlockInputStream or ECBlockReconstructedStripeInputStream - so we may move this logic around then, or change it.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,419 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  };
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }

Review comment:
       for (int i = 0; i < repConfig.getParity() - numRequired; i++) {
         indexes.remove(random.nextInt(indexes.size()));
       }
   
   Just another thought: Something like this can achieve the below logic? Probably we should change from ArrayList to LinkedList to reduce random location remove cost.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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


   @sodonnel Thanks for the nice work on this. I have reviewed the changes. The approach looks great to me. I have dropped few (initial)comments ( they are mostly minors). 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 a change in pull request #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */

Review comment:
       I see. Thanks for pointing it.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,419 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  };
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }

Review comment:
       for (int i = 0; i < repConfig.getParity() - numRequired; i++) {
         indexes.remove(r.nextInt(indexes.size()));
       }
   
   Just another thought: Something like this can achieve the below logic? Probably we should change from ArrayList to LinkedList to reduce random location remove cost.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */

Review comment:
       I think there is one case missing, i.e,  partial stripe with first chunk has less than ecChunkSize? does this case covered? I might be missing please point it to me.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */

Review comment:
       ```
    public void testReadPartialStripe() throws IOException {
       int blockLength = repConfig.getEcChunkSize() - 1;
   ```
   Should cover this - it sets up a block with a length of ecChunkSize - 1.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */
+public class TestECBlockReconstructedStripeInputStream {
+
+
+  private static final int ONEMB = 1024 * 1024;
+
+  private ECReplicationConfig repConfig;
+  private TestBlockInputStreamFactory streamFactory;
+
+  @Before
+  public void setup() {
+    repConfig = new ECReplicationConfig(3, 2,
+        ECReplicationConfig.EcCodec.RS, ONEMB);
+    streamFactory = new TestBlockInputStreamFactory();
+  }
+
+  @Test
+  public void testSufficientLocations() {
+    // One chunk, only 1 location.
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 1, ONEMB);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+    // Two Chunks, but missing data block 2.
+    dnMap = createIndexMap(1, 4, 5);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 2, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3.
+    dnMap = createIndexMap(1, 4, 5);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3 and parity 1.
+    dnMap = createIndexMap(1, 4);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+  }
+
+  @Test
+  public void testReadFullStripesWithPartial() throws IOException {
+    // Generate the input data for 3 full stripes and generate the parity.
+    int chunkSize = repConfig.getEcChunkSize();
+    int partialStripeSize = chunkSize * 2 - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+    dataBufs[1].limit(4 * chunkSize - 1);
+    dataBufs[2].limit(3 * chunkSize);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(createIndexMap(1, 4, 5));
+    // One data missing
+    locations.add(createIndexMap(1, 2, 4, 5));
+    // Two data missing including first
+    locations.add(createIndexMap(2, 4, 5));
+    // One data and one parity missing
+    locations.add(createIndexMap(2, 3, 4));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+
+      OmKeyLocationInfo keyInfo = createKeyInfo(repConfig,
+          stripeSize() * 3 + partialStripeSize, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+      try (ECBlockReconstructedStripeInputStream ecb =
+          new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+              null, null, streamFactory)) {
+        // Read 3 full stripes
+        for (int i = 0; i < 3; i++) {
+          int read = ecb.readStripe(bufs);
+          for (int j = 0; j < bufs.length; j++) {
+            validateContents(dataBufs[j], bufs[j], i * chunkSize, chunkSize);
+          }
+          Assert.assertEquals(stripeSize(), read);
+
+          // Check the underlying streams have read 1 chunk per read:
+          for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+            Assert.assertEquals(chunkSize * (i + 1),
+                bis.getPos());
+          }
+          Assert.assertEquals(stripeSize() * (i + 1), ecb.getPos());
+          clearBuffers(bufs);
+        }
+        // The next read is a partial stripe
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(partialStripeSize, read);
+        validateContents(dataBufs[0], bufs[0], 3 * chunkSize, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 3 * chunkSize, chunkSize - 1);
+        Assert.assertEquals(0, bufs[2].remaining());
+        Assert.assertEquals(0, bufs[2].position());
+
+        // A further read should give EOF
+        clearBuffers(bufs);
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  @Test
+  public void testReadPartialStripe() throws IOException {
+    int blockLength = repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(blockLength);
+    dataBufs[1].limit(0);
+    dataBufs[2].limit(0);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap = createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    try (ECBlockReconstructedStripeInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+            null, null, streamFactory)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      validateContents(dataBufs[0], bufs[0], 0, blockLength);
+      Assert.assertEquals(0, bufs[1].remaining());
+      Assert.assertEquals(0, bufs[1].position());
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 blockLength:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(blockLength, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeTwoChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 2 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(chunkSize);
+    dataBufs[1].limit(chunkSize - 1);
+    dataBufs[2].limit(0);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap = createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    try (ECBlockReconstructedStripeInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+            null, null, streamFactory)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      validateContents(dataBufs[0], bufs[0], 0, chunkSize);
+      validateContents(dataBufs[1], bufs[1], 0, chunkSize - 1);
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 chunk:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(chunkSize, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeThreeChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 3 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(chunkSize);
+    dataBufs[1].limit(chunkSize);
+    dataBufs[2].limit(chunkSize - 1);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    // We have a length that is less than a stripe, so chunks 1 and 2 are full.
+    // Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(createIndexMap(3, 4, 5));
+    // Two data missing
+    locations.add(createIndexMap(1, 4, 5));
+    // One data missing - the last one
+    locations.add(createIndexMap(1, 2, 5));
+    // One data and one parity missing
+    locations.add(createIndexMap(2, 3, 4));
+    // One data and one parity missing
+    locations.add(createIndexMap(1, 2, 4));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+
+      OmKeyLocationInfo keyInfo =
+          createKeyInfo(repConfig, blockLength, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+      try (ECBlockReconstructedStripeInputStream ecb =
+          new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+              null, null, streamFactory)) {
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(blockLength, read);
+        validateContents(dataBufs[0], bufs[0], 0, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 0, chunkSize);
+        validateContents(dataBufs[2], bufs[2], 0, chunkSize - 1);
+        // Check the underlying streams have been advanced by 1 chunk:
+        for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+          Assert.assertEquals(0, bis.getRemaining());
+        }
+        Assert.assertEquals(ecb.getPos(), blockLength);
+        clearBuffers(bufs);
+        // A further read should give EOF
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  private void addDataStreamsToFactory(ByteBuffer[] data, ByteBuffer[] parity) {
+    List<ByteBuffer> dataStreams = new ArrayList<>();
+    for (ByteBuffer b : data) {
+      dataStreams.add(b);
+    }
+    for (ByteBuffer b : parity) {
+      dataStreams.add(b);
+    }
+    streamFactory.setBlockStreamData(dataStreams);
+  }
+
+  /**
+   * Validates that the data buffer has the same contents as the source buffer,
+   * starting the checks in the src at offset and for count bytes.
+   * @param src The source of the data
+   * @param data The data which should be checked against the source
+   * @param offset The starting point in the src buffer
+   * @param count How many bytes to check.
+   */
+  private void validateContents(ByteBuffer src, ByteBuffer data, int offset,
+      int count) {
+    byte[] srcArray = src.array();
+    Assert.assertEquals(count, data.remaining());
+    for (int i = offset; i < offset + count; i++) {
+      Assert.assertEquals("Element " + i, srcArray[i], data.get());
+    }
+    data.flip();
+  }
+
+  /**
+   * Returns a new map containing a random DatanodeDetails for each index in
+   * inputs.
+   * @param idxs A list of indexes to add to the map
+   * @return A map of DatanodeDetails to index.
+   */
+  private Map<DatanodeDetails, Integer> createIndexMap(int... idxs) {
+    Map<DatanodeDetails, Integer> map = new HashMap<>();
+    for (int i : idxs) {
+      map.put(MockDatanodeDetails.randomDatanodeDetails(), i);
+    }
+    return map;
+  }
+
+  /**
+   * Given a set of data buffers, generate the parity data for the inputs.
+   * @param data A set of data buffers
+   * @param ecConfig The ECReplicationConfig representing the scheme
+   * @return
+   * @throws IOException
+   */
+  private ByteBuffer[] generateParity(ByteBuffer[] data,
+      ECReplicationConfig ecConfig) throws IOException {
+    // First data buffer dictates the size
+    int cellSize = data[0].limit();
+    // Store the positions of the remaining data buffers so we can restore them
+    int[] dataLimits = new int[data.length];
+    for (int i=1; i<data.length; i++) {
+      dataLimits[i] = data[i].limit();
+      data[i].limit(cellSize);
+      zeroFill(data[i]);
+      data[i].flip();
+    }
+    ByteBuffer[] parity = new ByteBuffer[ecConfig.getParity()];
+    for (int i=0; i<ecConfig.getParity(); i++) {

Review comment:
       Nit: There are some places code formatting off. Example above line. 




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */
+public class TestECBlockReconstructedStripeInputStream {
+
+
+  private static final int ONEMB = 1024 * 1024;
+
+  private ECReplicationConfig repConfig;
+  private TestBlockInputStreamFactory streamFactory;
+
+  @Before
+  public void setup() {
+    repConfig = new ECReplicationConfig(3, 2,
+        ECReplicationConfig.EcCodec.RS, ONEMB);
+    streamFactory = new TestBlockInputStreamFactory();
+  }
+
+  @Test
+  public void testSufficientLocations() {
+    // One chunk, only 1 location.
+    OmKeyLocationInfo keyInfo = createKeyInfo(repConfig, 1, ONEMB);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+    // Two Chunks, but missing data block 2.
+    dnMap = createIndexMap(1, 4, 5);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 2, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3.
+    dnMap = createIndexMap(1, 4, 5);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertTrue(ecb.hasSufficientLocations());
+    }
+
+    // Three Chunks, but missing data block 2 and 3 and parity 1.
+    dnMap = createIndexMap(1, 4);
+    keyInfo = createKeyInfo(repConfig, ONEMB * 3, dnMap);
+    try (ECBlockInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig,
+        keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+      Assert.assertFalse(ecb.hasSufficientLocations());
+    }
+  }
+
+  @Test
+  public void testReadFullStripesWithPartial() throws IOException {
+    // Generate the input data for 3 full stripes and generate the parity.
+    int chunkSize = repConfig.getEcChunkSize();
+    int partialStripeSize = chunkSize * 2 - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+    dataBufs[1].limit(4 * chunkSize - 1);
+    dataBufs[2].limit(3 * chunkSize);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(createIndexMap(1, 4, 5));
+    // One data missing
+    locations.add(createIndexMap(1, 2, 4, 5));
+    // Two data missing including first
+    locations.add(createIndexMap(2, 4, 5));
+    // One data and one parity missing
+    locations.add(createIndexMap(2, 3, 4));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+
+      OmKeyLocationInfo keyInfo = createKeyInfo(repConfig,
+          stripeSize() * 3 + partialStripeSize, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+      try (ECBlockReconstructedStripeInputStream ecb =
+          new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+              null, null, streamFactory)) {
+        // Read 3 full stripes
+        for (int i = 0; i < 3; i++) {
+          int read = ecb.readStripe(bufs);
+          for (int j = 0; j < bufs.length; j++) {
+            validateContents(dataBufs[j], bufs[j], i * chunkSize, chunkSize);
+          }
+          Assert.assertEquals(stripeSize(), read);
+
+          // Check the underlying streams have read 1 chunk per read:
+          for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+            Assert.assertEquals(chunkSize * (i + 1),
+                bis.getPos());
+          }
+          Assert.assertEquals(stripeSize() * (i + 1), ecb.getPos());
+          clearBuffers(bufs);
+        }
+        // The next read is a partial stripe
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(partialStripeSize, read);
+        validateContents(dataBufs[0], bufs[0], 3 * chunkSize, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 3 * chunkSize, chunkSize - 1);
+        Assert.assertEquals(0, bufs[2].remaining());
+        Assert.assertEquals(0, bufs[2].position());
+
+        // A further read should give EOF
+        clearBuffers(bufs);
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  @Test
+  public void testReadPartialStripe() throws IOException {
+    int blockLength = repConfig.getEcChunkSize() - 1;
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(blockLength);
+    dataBufs[1].limit(0);
+    dataBufs[2].limit(0);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap = createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    try (ECBlockReconstructedStripeInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+            null, null, streamFactory)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      validateContents(dataBufs[0], bufs[0], 0, blockLength);
+      Assert.assertEquals(0, bufs[1].remaining());
+      Assert.assertEquals(0, bufs[1].position());
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 blockLength:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(blockLength, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeTwoChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 2 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(chunkSize);
+    dataBufs[1].limit(chunkSize - 1);
+    dataBufs[2].limit(0);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+    addDataStreamsToFactory(dataBufs, parity);
+
+    ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+    // We have a length that is less than a single chunk, so blocks 2 and 3
+    // are padding and will not be present. Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+    Map<DatanodeDetails, Integer> dnMap = createIndexMap(4, 5);
+    OmKeyLocationInfo keyInfo =
+        createKeyInfo(repConfig, blockLength, dnMap);
+    streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+    try (ECBlockReconstructedStripeInputStream ecb =
+        new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+            null, null, streamFactory)) {
+      int read = ecb.readStripe(bufs);
+      Assert.assertEquals(blockLength, read);
+      validateContents(dataBufs[0], bufs[0], 0, chunkSize);
+      validateContents(dataBufs[1], bufs[1], 0, chunkSize - 1);
+      Assert.assertEquals(0, bufs[2].remaining());
+      Assert.assertEquals(0, bufs[2].position());
+      // Check the underlying streams have been advanced by 1 chunk:
+      for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+        Assert.assertEquals(chunkSize, bis.getPos());
+      }
+      Assert.assertEquals(ecb.getPos(), blockLength);
+      clearBuffers(bufs);
+      // A further read should give EOF
+      read = ecb.readStripe(bufs);
+      Assert.assertEquals(-1, read);
+    }
+  }
+
+  @Test
+  public void testReadPartialStripeThreeChunks() throws IOException {
+    int chunkSize = repConfig.getEcChunkSize();
+    int blockLength = chunkSize * 3 - 1;
+
+    ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+    // First buffer has only the blockLength, the other two will have no data.
+    dataBufs[0].limit(chunkSize);
+    dataBufs[1].limit(chunkSize);
+    dataBufs[2].limit(chunkSize - 1);
+    for (ByteBuffer b : dataBufs) {
+      randomFill(b);
+    }
+    ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+    // We have a length that is less than a stripe, so chunks 1 and 2 are full.
+    // Block 1 is lost and needs recovered
+    // from the parity and padded blocks 2 and 3.
+
+    List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+    // Two data missing
+    locations.add(createIndexMap(3, 4, 5));
+    // Two data missing
+    locations.add(createIndexMap(1, 4, 5));
+    // One data missing - the last one
+    locations.add(createIndexMap(1, 2, 5));
+    // One data and one parity missing
+    locations.add(createIndexMap(2, 3, 4));
+    // One data and one parity missing
+    locations.add(createIndexMap(1, 2, 4));
+
+    for (Map<DatanodeDetails, Integer> dnMap : locations) {
+      streamFactory = new TestBlockInputStreamFactory();
+      addDataStreamsToFactory(dataBufs, parity);
+      ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+
+      OmKeyLocationInfo keyInfo =
+          createKeyInfo(repConfig, blockLength, dnMap);
+      streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+      try (ECBlockReconstructedStripeInputStream ecb =
+          new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+              null, null, streamFactory)) {
+        int read = ecb.readStripe(bufs);
+        Assert.assertEquals(blockLength, read);
+        validateContents(dataBufs[0], bufs[0], 0, chunkSize);
+        validateContents(dataBufs[1], bufs[1], 0, chunkSize);
+        validateContents(dataBufs[2], bufs[2], 0, chunkSize - 1);
+        // Check the underlying streams have been advanced by 1 chunk:
+        for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+          Assert.assertEquals(0, bis.getRemaining());
+        }
+        Assert.assertEquals(ecb.getPos(), blockLength);
+        clearBuffers(bufs);
+        // A further read should give EOF
+        read = ecb.readStripe(bufs);
+        Assert.assertEquals(-1, read);
+      }
+    }
+  }
+
+  private void addDataStreamsToFactory(ByteBuffer[] data, ByteBuffer[] parity) {
+    List<ByteBuffer> dataStreams = new ArrayList<>();
+    for (ByteBuffer b : data) {
+      dataStreams.add(b);
+    }
+    for (ByteBuffer b : parity) {
+      dataStreams.add(b);
+    }
+    streamFactory.setBlockStreamData(dataStreams);
+  }
+
+  /**
+   * Validates that the data buffer has the same contents as the source buffer,
+   * starting the checks in the src at offset and for count bytes.
+   * @param src The source of the data
+   * @param data The data which should be checked against the source
+   * @param offset The starting point in the src buffer
+   * @param count How many bytes to check.
+   */
+  private void validateContents(ByteBuffer src, ByteBuffer data, int offset,
+      int count) {
+    byte[] srcArray = src.array();
+    Assert.assertEquals(count, data.remaining());
+    for (int i = offset; i < offset + count; i++) {
+      Assert.assertEquals("Element " + i, srcArray[i], data.get());
+    }
+    data.flip();
+  }
+
+  /**
+   * Returns a new map containing a random DatanodeDetails for each index in
+   * inputs.
+   * @param idxs A list of indexes to add to the map
+   * @return A map of DatanodeDetails to index.
+   */
+  private Map<DatanodeDetails, Integer> createIndexMap(int... idxs) {
+    Map<DatanodeDetails, Integer> map = new HashMap<>();
+    for (int i : idxs) {
+      map.put(MockDatanodeDetails.randomDatanodeDetails(), i);
+    }
+    return map;
+  }
+
+  /**
+   * Given a set of data buffers, generate the parity data for the inputs.
+   * @param data A set of data buffers
+   * @param ecConfig The ECReplicationConfig representing the scheme
+   * @return
+   * @throws IOException
+   */
+  private ByteBuffer[] generateParity(ByteBuffer[] data,
+      ECReplicationConfig ecConfig) throws IOException {
+    // First data buffer dictates the size
+    int cellSize = data[0].limit();
+    // Store the positions of the remaining data buffers so we can restore them
+    int[] dataLimits = new int[data.length];
+    for (int i=1; i<data.length; i++) {
+      dataLimits[i] = data[i].limit();
+      data[i].limit(cellSize);
+      zeroFill(data[i]);
+      data[i].flip();
+    }
+    ByteBuffer[] parity = new ByteBuffer[ecConfig.getParity()];
+    for (int i=0; i<ecConfig.getParity(); i++) {

Review comment:
       Yeah. I am not worried either way, but they should be consistent across. So that formatting will not introduce unwanted changes if someone does format later.
   Thanks for fixing. I remember check style default rules should point that, may be some one disable as it might be giving too many comments( I am not sure)?




-- 
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 merged pull request #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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


   


-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */

Review comment:
       ``` public void testReadPartialStripe() throws IOException {
       int blockLength = repConfig.getEcChunkSize() - 1;
   ```
   Should cover this - it sets up a block with a length of ecChunkSize - 1.




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,419 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  };

Review comment:
       Remove unnecessary ';'




-- 
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 #2797: HDDS-5551. EC: Implement an Input Stream to reconstruct EC blocks on demand

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {

Review comment:
       Sure. 

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();

Review comment:
       +1

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
##########
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ *     readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+  // List of buffers, data + parity long, needed by the EC decoder. Missing
+  // or not-need locations will be null.
+  private ByteBuffer[] decoderInputBuffers;
+  // Missing chunks are recovered into these buffers.
+  private ByteBuffer[] decoderOutputBuffers;
+  // Missing indexes to be recovered into the recovered buffers. Required by the
+  // EC decoder
+  private int[] missingIndexes;
+  // The blockLocation indexes to use to read data into the dataBuffers.
+  private List<Integer> dataIndexes = new ArrayList<>();
+
+  private final RawErasureDecoder decoder;
+
+  private boolean initialized = false;
+
+  public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+      OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+       XceiverClientFactory xceiverClientFactory, Function<BlockID,
+      Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+    super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+        refreshFunction, streamFactory);
+
+    decoder = CodecRegistry.getInstance()
+        .getCodecFactory(repConfig.getCodec().toString())
+        .createDecoder(repConfig);
+  }
+
+  protected void init() throws InsufficientLocationsException {
+    if (!hasSufficientLocations()) {
+      throw new InsufficientLocationsException("There are not enough " +
+          "datanodes to read the EC block");
+    }
+
+    ECReplicationConfig repConfig = getRepConfig();
+    // The EC decoder needs an array data+parity long, with missing or not
+    // needed indexes set to null.
+    decoderInputBuffers = new ByteBuffer[
+        getRepConfig().getData() + getRepConfig().getParity()];
+    DatanodeDetails[] locations = getDataLocations();
+    setMissingIndexesAndDataLocations(locations);
+    List<Integer> parityIndexes =
+        selectParityIndexes(locations, missingIndexes.length);
+    // We read from the selected parity blocks, so add them to the data indexes.
+    dataIndexes.addAll(parityIndexes);
+    // The decoder inputs originally start as all nulls. Then we populate the
+    // pieces we have data for. The parity buffers are reused for the block
+    // so we can allocated them now.
+    for (Integer i : parityIndexes) {
+      decoderInputBuffers[i] = allocateBuffer(repConfig);
+    }
+    decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+    initialized = true;
+  }
+
+  /**
+   * Determine which indexes are missing, taking into account the length of the
+   * block. For a block shorter than a full EC stripe, it is expected that
+   * some of the data locations will not be present.
+   * Populates the missingIndex and dataIndexes instance variables.
+   * @param locations Available locations for the block group
+   */
+  private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+    ECReplicationConfig repConfig = getRepConfig();
+    int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+    List<Integer> missingInd = new ArrayList<>();
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (locations[i] == null && i < expectedDataBlocks) {
+        missingInd.add(i);
+      } else if (locations[i] != null) {
+        dataIndexes.add(i);
+      }
+    }
+    missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+  }
+
+  private void assignBuffers(ByteBuffer[] bufs) {
+    ECReplicationConfig repConfig = getRepConfig();
+    Preconditions.assertTrue(bufs.length == repConfig.getData());
+    int recoveryIndex = 0;
+    // Here bufs come from the caller and will be filled with data read from
+    // the blocks or recovered. Therefore, if the index is missing, we assign
+    // the buffer to the decoder outputs, where data is recovered via EC
+    // decoding. Otherwise the buffer is set to the input. Note, it may be a
+    // buffer which needs padded.
+    for (int i = 0; i < repConfig.getData(); i++) {
+      if (isMissingIndex(i)) {
+        decoderOutputBuffers[recoveryIndex++] = bufs[i];
+      } else {
+        decoderInputBuffers[i] = bufs[i];
+      }
+    }
+  }
+
+  private boolean isMissingIndex(int ind) {
+    for (int i : missingIndexes) {
+      if (i == ind) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This method should be passed a list of byteBuffers which must contain EC
+   * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+   * ChunkSize bytes remaining. After returning, the buffers will contain the
+   * data for the next stripe in the block. The buffers will be returned
+   * "ready to read" with their position set to zero and the limit set
+   * according to how much data they contain.
+   *
+   * @param bufs A list of byteBuffers which must contain EC Data Number
+   *             entries. Each Bytebuffer should be at position 0 and have
+   *             EC ChunkSize bytes remaining.
+   *
+   * @return The number of bytes read
+   * @throws IOException
+   */
+  public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+    if (!initialized) {
+      init();
+    }
+    int toRead = (int)Math.min(getRemaining(), getStripeSize());
+    if (toRead == 0) {
+      return EOF;
+    }
+    validateBuffers(bufs);
+    assignBuffers(bufs);
+    clearParityBuffers();
+    loadDataBuffersFromStream();
+    padBuffers(toRead);
+    flipInputs();
+    decodeStripe();
+    unPadBuffers(bufs, toRead);
+    setPos(getPos() + toRead);
+    return toRead;
+  }
+
+  private void validateBuffers(ByteBuffer[] bufs) {
+    Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+    int chunkSize = getRepConfig().getEcChunkSize();
+    for (ByteBuffer b : bufs) {
+      Preconditions.assertTrue(b.remaining() == chunkSize);
+    }
+  }
+
+  private void padBuffers(int toRead) {
+    int dataNum = getRepConfig().getData();
+    int parityNum = getRepConfig().getParity();
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    if (fullChunks == dataNum) {
+      // There is no padding to do - we are reading a full stripe.
+      return;
+    }
+    // The size of each chunk is governed by the size of the first chunk.
+    // The parity always matches the first chunk size.
+    int paritySize = Math.min(toRead, chunkSize);
+    // We never need to pad the first chunk - its length dictates the length
+    // of all others.
+    fullChunks = Math.max(1, fullChunks);
+    for (int i = fullChunks; i < dataNum; i++) {
+      ByteBuffer buf = decoderInputBuffers[i];
+      if (buf != null) {
+        buf.limit(paritySize);
+        zeroFill(buf);
+      }
+    }
+    // Ensure the available parity buffers are the expected length
+    for (int i = dataNum; i < dataNum + parityNum; i++) {
+      ByteBuffer b = decoderInputBuffers[i];
+      if (b != null) {
+        Preconditions.assertTrue(b.position() == paritySize);
+      }
+    }
+    // The output buffers need their limit set to the parity size
+    for (ByteBuffer b : decoderOutputBuffers) {
+      b.limit(paritySize);
+    }
+  }
+
+  private void unPadBuffers(ByteBuffer[] bufs, int toRead) {
+    int chunkSize = getRepConfig().getEcChunkSize();
+    int fullChunks = toRead / chunkSize;
+    int remainingLength = toRead % chunkSize;
+    if (fullChunks == getRepConfig().getData()) {
+      // We are reading a full stripe, no concerns over padding.
+      return;
+    }
+
+    if (fullChunks == 0){
+      // All buffers except the first contain no data.
+      for (int i = 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    } else {
+      // The first partial has the remaining length
+      bufs[fullChunks].limit(remainingLength);
+      // All others have a zero limit
+      for (int i = fullChunks + 1; i < bufs.length; i++) {
+        bufs[i].position(0);
+        bufs[i].limit(0);
+      }
+    }
+  }
+
+  private void zeroFill(ByteBuffer buf) {
+    // fill with zeros from pos to limit.
+    if (buf.hasArray()) {
+      byte[] a = buf.array();
+      Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+      buf.position(buf.limit());
+    } else {
+      while (buf.hasRemaining()) {
+        buf.put((byte)0);
+      }
+    }
+  }
+
+  /**
+   * Take the parity indexes which are available, shuffle them and truncate the
+   * list to the number of required parity chunks.
+   * @param locations The list of locations for all blocks in the block group/
+   * @param numRequired The number of parity chunks needed for reconstruction
+   * @return A list of indexes indicating which parity locations to read.
+   */
+  private List<Integer> selectParityIndexes(
+      DatanodeDetails[] locations, int numRequired) {
+    List<Integer> indexes = new ArrayList<>();
+    ECReplicationConfig repConfig = getRepConfig();
+    for (int i = repConfig.getData();
+         i < repConfig.getParity() + repConfig.getData(); i++) {
+      if (locations[i] != null) {
+        indexes.add(i);
+      }
+    }
+    Preconditions.assertTrue(indexes.size() >= numRequired);
+    Random rand = new Random();
+    while (indexes.size() > numRequired) {
+      indexes.remove(rand.nextInt(indexes.size()));
+    }
+    return indexes;
+  }
+
+  private ByteBuffer allocateBuffer(ECReplicationConfig repConfig) {
+    ByteBuffer buf = ByteBuffer.allocate(repConfig.getEcChunkSize());
+    return buf;
+  }
+
+  private void flipInputs() {
+    for (ByteBuffer b : decoderInputBuffers) {
+      if (b != null) {
+        b.flip();
+      }
+    }
+  }
+
+  private void clearParityBuffers() {
+    for (int i = getRepConfig().getData();
+         i < getRepConfig().getRequiredNodes(); i++) {
+      if (decoderInputBuffers[i] != null) {
+        decoderInputBuffers[i].clear();
+      }
+    }
+  }
+
+  protected void loadDataBuffersFromStream() throws IOException {
+    for (int i = 0; i < dataIndexes.size(); i++) {
+      BlockExtendedInputStream stream =
+          getOrOpenStream(i, dataIndexes.get(i));
+      ByteBuffer b = decoderInputBuffers[dataIndexes.get(i)];
+      while (b.hasRemaining()) {
+        int read = stream.read(b);
+        if (read == EOF) {
+          break;
+        }
+      }
+    }
+  }
+
+  /**
+   * Take the populated input buffers and missing indexes and create the
+   * outputs. Note that the input buffers have to be "ready for read", ie they
+   * need to have been flipped after their data was loaded. The created outputs
+   * are "ready to read" by the underlying decoder API, so there is no need to
+   * flip them after the call. The decoder reads all the inputs leaving the
+   * buffer position at the end, so the inputs are flipped after the decode so
+   * we have a complete set of "outputs" for the EC Stripe which are ready to
+   * read.
+   * @throws IOException
+   */
+  private void decodeStripe() throws IOException {
+    decoder.decode(decoderInputBuffers, missingIndexes, decoderOutputBuffers);
+    flipInputs();
+  }
+
+  @Override

Review comment:
       ok. 




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