Uploaded image for project: 'Apache Ozone'
  1. Apache Ozone
  2. HDDS-3816 Erasure Coding
  3. HDDS-6342

EC: Fix large write with multiple stripes upon stripe failure.

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • EC-Branch
    • None

    Description

      Test with ockg

      ./bin/ozone freon ockg -p test -n 50 -t 8 -s $((500*1024*1024)) --type=EC --replication=rs-10-4-1024k

      2022-02-15 12:43:11,295 [pool-2-thread-7] ERROR freon.BaseFreonGenerator: Error on executing task 46
      java.lang.IllegalArgumentException
              at com.google.common.base.Preconditions.checkArgument(Preconditions.java:130)
              at org.apache.hadoop.ozone.client.io.BlockOutputStreamEntryPool.commitKey(BlockOutputStreamEntryPool.java:327)
              at org.apache.hadoop.ozone.client.io.ECKeyOutputStream.close(ECKeyOutputStream.java:536)
              at org.apache.hadoop.ozone.client.io.OzoneOutputStream.close(OzoneOutputStream.java:61)
              at org.apache.hadoop.ozone.freon.OzoneClientKeyGenerator.lambda$createKey$36(OzoneClientKeyGenerator.java:150)
              at com.codahale.metrics.Timer.time(Timer.java:101)
              at org.apache.hadoop.ozone.freon.OzoneClientKeyGenerator.createKey(OzoneClientKeyGenerator.java:142)
              at org.apache.hadoop.ozone.freon.BaseFreonGenerator.tryNextTask(BaseFreonGenerator.java:183)
              at org.apache.hadoop.ozone.freon.BaseFreonGenerator.taskLoop(BaseFreonGenerator.java:163)
              at org.apache.hadoop.ozone.freon.BaseFreonGenerator.lambda$startTaskRunners$1(BaseFreonGenerator.java:146)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748) 

      This happens only when write happen failure during parity write and there are > 1 already written stripes in the current block group.

      Upon this a new block group is picked for retrying the current stripe write, and the current block group should rollback its current position, the bug lies within the calculation of the acked length of the block group.

      Code references:

      if (handleParityWrites(ecChunkSize, allocateBlockIfFull,
          shouldClose) == StripeWriteStatus.FAILED) {
        handleStripeFailure(numDataBlks * ecChunkSize, allocateBlockIfFull,
            shouldClose);
      } else {
        // At this stage stripe write is successful.
        currentStreamEntry.updateBlockGroupToAckedPosition(
            currentStreamEntry.getCurrentPosition());
      } 
      private StripeWriteStatus rewriteStripeToNewBlockGroup(
          int failedStripeDataSize, boolean allocateBlockIfFull, boolean close)
          throws IOException {
        long[] failedDataStripeChunkLens = new long[numDataBlks];
        long[] failedParityStripeChunkLens = new long[numParityBlks];
        final ByteBuffer[] dataBuffers = ecChunkBufferCache.getDataBuffers();
        for (int i = 0; i < numDataBlks; i++) {
          failedDataStripeChunkLens[i] = dataBuffers[i].limit();
        }
        final ByteBuffer[] parityBuffers = ecChunkBufferCache.getParityBuffers();
        for (int i = 0; i < numParityBlks; i++) {
          failedParityStripeChunkLens[i] = parityBuffers[i].limit();
        }
      
        blockOutputStreamEntryPool.getCurrentStreamEntry().resetToFirstEntry();
        // Rollback the length/offset updated as part of this failed stripe write.
        offset -= failedStripeDataSize;
        blockOutputStreamEntryPool.getCurrentStreamEntry()
            .resetToAckedPosition();                         <-- wrong position deteced
      ...
      } 

       

       

      Attachments

        Issue Links

          Activity

            People

              markgui Mark Gui
              markgui Mark Gui
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: