You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2018/01/04 20:25:31 UTC

[22/23] hadoop git commit: HDFS-12860. StripedBlockUtil#getRangesInternalBlocks throws exception for the block group size larger than 2GB. (Contributed by Lei (Eddy) Xu)

HDFS-12860. StripedBlockUtil#getRangesInternalBlocks throws exception for the block group size larger than 2GB. (Contributed by Lei (Eddy) Xu)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dc735b28
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dc735b28
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dc735b28

Branch: refs/heads/HDFS-7240
Commit: dc735b286bb656903df49aee776d22ee0c61f860
Parents: 739d3c3
Author: Lei Xu <le...@apache.org>
Authored: Thu Jan 4 10:16:40 2018 -0800
Committer: Lei Xu <le...@apache.org>
Committed: Thu Jan 4 10:54:56 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hdfs/util/StripedBlockUtil.java      | 63 +++++++++++++++-----
 .../hadoop/hdfs/util/TestStripedBlockUtil.java  | 41 ++++++++++++-
 2 files changed, 86 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc735b28/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
index 9e24576..9bad45d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -396,7 +396,9 @@ public class StripedBlockUtil {
       long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
     Preconditions.checkArgument(
         rangeStartInBlockGroup <= rangeEndInBlockGroup &&
-            rangeEndInBlockGroup < blockGroup.getBlockSize());
+            rangeEndInBlockGroup < blockGroup.getBlockSize(),
+        "start=%s end=%s blockSize=%s", rangeStartInBlockGroup,
+        rangeEndInBlockGroup, blockGroup.getBlockSize());
     long len = rangeEndInBlockGroup - rangeStartInBlockGroup + 1;
     int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
     int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize);
@@ -578,28 +580,39 @@ public class StripedBlockUtil {
   public static class StripingCell {
     final ErasureCodingPolicy ecPolicy;
     /** Logical order in a block group, used when doing I/O to a block group. */
-    final int idxInBlkGroup;
-    final int idxInInternalBlk;
-    final int idxInStripe;
+    private final long idxInBlkGroup;
+    private final long idxInInternalBlk;
+    private final int idxInStripe;
     /**
      * When a logical byte range is mapped to a set of cells, it might
      * partially overlap with the first and last cells. This field and the
      * {@link #size} variable represent the start offset and size of the
      * overlap.
      */
-    final int offset;
-    final int size;
+    private final long offset;
+    private final int size;
 
-    StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, int idxInBlkGroup,
-        int offset) {
+    StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, long idxInBlkGroup,
+        long offset) {
       this.ecPolicy = ecPolicy;
       this.idxInBlkGroup = idxInBlkGroup;
       this.idxInInternalBlk = idxInBlkGroup / ecPolicy.getNumDataUnits();
-      this.idxInStripe = idxInBlkGroup -
-          this.idxInInternalBlk * ecPolicy.getNumDataUnits();
+      this.idxInStripe = (int)(idxInBlkGroup -
+          this.idxInInternalBlk * ecPolicy.getNumDataUnits());
       this.offset = offset;
       this.size = cellSize;
     }
+
+    int getIdxInStripe() {
+      return idxInStripe;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("StripingCell(idxInBlkGroup=%d, " +
+          "idxInInternalBlk=%d, idxInStrip=%d, offset=%d, size=%d)",
+          idxInBlkGroup, idxInInternalBlk, idxInStripe, offset, size);
+    }
   }
 
   /**
@@ -646,7 +659,9 @@ public class StripedBlockUtil {
     public int missingChunksNum = 0;
 
     public AlignedStripe(long offsetInBlock, long length, int width) {
-      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0,
+          "OffsetInBlock(%s) and length(%s) must be non-negative",
+          offsetInBlock, length);
       this.range = new VerticalRange(offsetInBlock, length);
       this.chunks = new StripingChunk[width];
     }
@@ -665,9 +680,9 @@ public class StripedBlockUtil {
 
     @Override
     public String toString() {
-      return "Offset=" + range.offsetInBlock + ", length=" + range.spanInBlock +
-          ", fetchedChunksNum=" + fetchedChunksNum +
-          ", missingChunksNum=" + missingChunksNum;
+      return "AlignedStripe(Offset=" + range.offsetInBlock + ", length=" +
+          range.spanInBlock + ", fetchedChunksNum=" + fetchedChunksNum +
+          ", missingChunksNum=" + missingChunksNum + ")";
     }
   }
 
@@ -698,7 +713,9 @@ public class StripedBlockUtil {
     public long spanInBlock;
 
     public VerticalRange(long offsetInBlock, long length) {
-      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0,
+          "OffsetInBlock(%s) and length(%s) must be non-negative",
+          offsetInBlock, length);
       this.offsetInBlock = offsetInBlock;
       this.spanInBlock = length;
     }
@@ -707,6 +724,12 @@ public class StripedBlockUtil {
     public boolean include(long pos) {
       return pos >= offsetInBlock && pos < offsetInBlock + spanInBlock;
     }
+
+    @Override
+    public String toString() {
+      return String.format("VerticalRange(offsetInBlock=%d, spanInBlock=%d)",
+          this.offsetInBlock, this.spanInBlock);
+    }
   }
 
   /**
@@ -880,7 +903,9 @@ public class StripedBlockUtil {
     final long length;
 
     public StripeRange(long offsetInBlock, long length) {
-      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0,
+          "Offset(%s) and length(%s) must be non-negative", offsetInBlock,
+          length);
       this.offsetInBlock = offsetInBlock;
       this.length = length;
     }
@@ -892,6 +917,12 @@ public class StripedBlockUtil {
     public long getLength() {
       return length;
     }
+
+    @Override
+    public String toString() {
+      return String.format("StripeRange(offsetInBlock=%d, length=%d)",
+          offsetInBlock, length);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc735b28/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
index 5de63eb..fe6bb09 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
@@ -40,6 +40,7 @@ import java.util.Random;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Need to cover the following combinations:
@@ -127,7 +128,7 @@ public class TestStripedBlockUtil {
     return (byte) (((i + 13) * 29) & BYTE_MASK);
   }
 
-  private LocatedStripedBlock createDummyLocatedBlock(int bgSize) {
+  private LocatedStripedBlock createDummyLocatedBlock(long bgSize) {
     final long blockGroupID = -1048576;
     DatanodeInfo[] locs = new DatanodeInfo[groupSize];
     String[] storageIDs = new String[groupSize];
@@ -160,7 +161,7 @@ public class TestStripedBlockUtil {
       Preconditions.checkState(done % cellSize == 0);
       StripingCell cell =
           new StripingCell(ecPolicy, cellSize, done / cellSize, 0);
-      int idxInStripe = cell.idxInStripe;
+      int idxInStripe = cell.getIdxInStripe();
       int size = Math.min(cellSize, bgSize - done);
       for (int i = 0; i < size; i++) {
         bufs[idxInStripe][pos[idxInStripe] + i] = hashIntToByte(done + i);
@@ -283,4 +284,40 @@ public class TestStripedBlockUtil {
       }
     }
   }
+
+  /**
+   * Test dividing a byte range that located above the 2GB range, which is
+   * {@link Integer#MAX_VALUE}.
+   *
+   * HDFS-12860 occurs when {@link VerticalRange#offsetInBlock} is larger than
+   * {@link Integer#MAX_VALUE}
+   *
+   * Take RS-6-3-1024k EC policy as example:
+   *  <li>cellSize = 1MB</li>
+   *  <li>The first {@link VerticalRange#offsetInBlock} that is larger than
+   *  {@link Integer#MAX_VALUE} is Math.ceilInteger.MAX_VALUE / cellSize = 2048
+   *  </li>
+   *  <li>The first offset in block group that causes HDFS-12860 is:
+   *  2048 * cellSize * dataBlocks (6)</li>
+   */
+  @Test
+  public void testDivideOneStripeLargeBlockSize() {
+    ByteBuffer buffer = ByteBuffer.allocate(stripeSize);
+
+    // This offset will cause overflow before HDFS-12860.
+    long offsetInInternalBlk = Integer.MAX_VALUE / cellSize + 10;
+    long rangeStartInBlockGroup = offsetInInternalBlk * dataBlocks * cellSize;
+    long rangeEndInBlockGroup = rangeStartInBlockGroup +
+        dataBlocks / 2 * cellSize - 1;
+    // each block is 4GB, each block group has 4GB * (6 + 3) = 36GB.
+    long blockGroupSize = 4096L * cellSize * groupSize;
+    LocatedStripedBlock blockGroup = createDummyLocatedBlock(blockGroupSize);
+    AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(ecPolicy,
+        cellSize, blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup,
+        buffer);
+    long offset = offsetInInternalBlk * cellSize;
+    assertTrue(offset > Integer.MAX_VALUE);
+    assertEquals(offset, stripes[0].range.offsetInBlock);
+    assertEquals(1, stripes.length);
+  }
 }


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