You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by vi...@apache.org on 2018/06/22 21:14:38 UTC

[drill] 07/09: DRILL-6486: BitVector split and transfer does not work correctly for non byte-multiple transfer lengths

This is an automated email from the ASF dual-hosted git repository.

vitalii pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 947e6e561414182f2060d905daae197f3dd71f9f
Author: karthik <km...@maprtech.com>
AuthorDate: Thu Jun 7 13:23:04 2018 -0700

    DRILL-6486: BitVector split and transfer does not work correctly for non byte-multiple transfer lengths
    
    Fix for the bug in BitVector splitAndTransfer. The logic for handling copy of last-n bits was incorrect for none byte-multiple transfer lengths.
    
    closes #1316
---
 .../drill/exec/record/vector/TestValueVector.java  |   3 +
 .../drill/exec/vector/TestSplitAndTransfer.java    | 108 ++++++++++++++++++++-
 .../org/apache/drill/exec/vector/BitVector.java    |  45 ++++++---
 3 files changed, 141 insertions(+), 15 deletions(-)

diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
index 52a1585..64b8cae 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -548,6 +548,7 @@ the interface to load has changed
       m.set(1, 0);
       m.set(100, 0);
       m.set(1022, 1);
+      m.setValueCount(1023);
 
       final BitVector.Accessor accessor = vector.getAccessor();
       assertEquals(1, accessor.get(0));
@@ -560,12 +561,14 @@ the interface to load has changed
       m.set(0, 1);
       m.set(1, 0);
       m.set(1, 0);
+      m.setValueCount(2);
       assertEquals(1, accessor.get(0));
       assertEquals(0, accessor.get(1));
 
       // test toggling the values
       m.set(0, 0);
       m.set(1, 1);
+      m.setValueCount(2);
       assertEquals(0, accessor.get(0));
       assertEquals(1, accessor.get(1));
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/TestSplitAndTransfer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/TestSplitAndTransfer.java
index 1553a9a..057fa13 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/TestSplitAndTransfer.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/TestSplitAndTransfer.java
@@ -17,10 +17,6 @@
  */
 package org.apache.drill.exec.vector;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
@@ -31,6 +27,12 @@ import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.vector.NullableVarCharVector.Accessor;
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertArrayEquals;
+
+
 public class TestSplitAndTransfer {
   @Test
   public void test() throws Exception {
@@ -77,4 +79,102 @@ public class TestSplitAndTransfer {
     varCharVector.close();
     allocator.close();
   }
+
+  /**
+   *  BitVector tests
+   */
+
+  enum TestBitPattern {
+    ZERO,
+    ONE,
+    ALTERNATING,
+    RANDOM
+  }
+
+  @Test
+  public void testBitVectorUnalignedStart() throws Exception {
+
+    testBitVectorImpl(24, new int[][] {{5, 17}}, TestBitPattern.ONE);
+    testBitVectorImpl(24, new int[][] {{5, 17}}, TestBitPattern.ZERO);
+    testBitVectorImpl(24, new int[][] {{5, 17}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(24, new int[][] {{5, 17}}, TestBitPattern.RANDOM);
+
+    testBitVectorImpl(3443, new int[][] {{0, 2047}, {2047, 1396}}, TestBitPattern.ZERO);
+    testBitVectorImpl(3443, new int[][] {{0, 2047}, {2047, 1396}}, TestBitPattern.ONE);
+    testBitVectorImpl(3443, new int[][] {{0, 2047}, {2047, 1396}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(3443, new int[][] {{0, 2047}, {2047, 1396}}, TestBitPattern.RANDOM);
+
+    testBitVectorImpl(3447, new int[][] {{0, 2047}, {2047, 1400}}, TestBitPattern.ZERO);
+    testBitVectorImpl(3447, new int[][] {{0, 2047}, {2047, 1400}}, TestBitPattern.ONE);
+    testBitVectorImpl(3447, new int[][] {{0, 2047}, {2047, 1400}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(3447, new int[][] {{0, 2047}, {2047, 1400}}, TestBitPattern.RANDOM);
+  }
+
+  @Test
+  public void testBitVectorAlignedStart() throws Exception {
+
+    testBitVectorImpl(24, new int[][] {{0, 17}}, TestBitPattern.ONE);
+    testBitVectorImpl(24, new int[][] {{0, 17}}, TestBitPattern.ZERO);
+    testBitVectorImpl(24, new int[][] {{0, 17}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(24, new int[][] {{0, 17}}, TestBitPattern.RANDOM);
+
+    testBitVectorImpl(3444, new int[][] {{0, 2048}, {2048, 1396}}, TestBitPattern.ZERO);
+    testBitVectorImpl(3444, new int[][] {{0, 2048}, {2048, 1396}}, TestBitPattern.ONE);
+    testBitVectorImpl(3444, new int[][] {{0, 2048}, {2048, 1396}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(3444, new int[][] {{0, 2048}, {2048, 1396}}, TestBitPattern.RANDOM);
+
+    testBitVectorImpl(3448, new int[][] {{0, 2048}, {2048, 1400}}, TestBitPattern.ZERO);
+    testBitVectorImpl(3448, new int[][] {{0, 2048}, {2048, 1400}}, TestBitPattern.ONE);
+    testBitVectorImpl(3448, new int[][] {{0, 2048}, {2048, 1400}}, TestBitPattern.ALTERNATING);
+    testBitVectorImpl(3448, new int[][] {{0, 2048}, {2048, 1400}}, TestBitPattern.RANDOM);
+  }
+
+  int getBit(TestBitPattern pattern, int index) {
+    if (pattern == TestBitPattern.RANDOM) {
+      return (int) (Math.random() * 2);
+    }
+    return (pattern == TestBitPattern.ALTERNATING) ? (index % 2) : ((pattern == TestBitPattern.ONE) ? 1 : 0);
+  }
+
+  public void testBitVectorImpl(int valueCount, final int[][] startLengths, TestBitPattern pattern) throws Exception {
+    final DrillConfig drillConfig = DrillConfig.create();
+    final BufferAllocator allocator = RootAllocatorFactory.newRoot(drillConfig);
+    final MaterializedField field = MaterializedField.create("field", Types.optional(MinorType.BIT));
+    final BitVector bitVector = new BitVector(field, allocator);
+    bitVector.allocateNew(valueCount  + 8); // extra byte at the end that gets filled with junk
+    final int[] compareArray = new int[valueCount];
+
+    int testBitValue = 0 ;
+    final BitVector.Mutator mutator = bitVector.getMutator();
+    for (int i = 0; i < valueCount; i ++) {
+      testBitValue = getBit(pattern, i);
+      mutator.set(i, testBitValue);
+      compareArray[i] = testBitValue;
+    }
+
+    // write some junk value at the end to catch
+    // off-by-one out-of-bound reads
+    for (int j = valueCount; j < valueCount + 8; j++) {
+      mutator.set(j, ~testBitValue); // fill with compliment of testBit
+    }
+    mutator.setValueCount(valueCount);
+
+    final TransferPair tp = bitVector.getTransferPair(allocator);
+    final BitVector newBitVector = (BitVector) tp.getTo();
+    final BitVector.Accessor accessor = newBitVector.getAccessor();
+
+    for (final int[] startLength : startLengths) {
+      final int start = startLength[0];
+      final int length = startLength[1];
+      tp.splitAndTransfer(start, length);
+      assertEquals(newBitVector.getAccessor().getValueCount(), length);
+      for (int i = 0; i < length; i++) {
+        final int expectedValue = compareArray[start + i];
+        assertEquals(expectedValue, accessor.get(i));
+      }
+      newBitVector.clear();
+    }
+    bitVector.close();
+    allocator.close();
+  }
 }
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
index 3725364..0dd34f5 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -285,20 +285,20 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
 
   public void splitAndTransferTo(int startIndex, int length, BitVector target) {
     assert startIndex + length <= valueCount;
-    int firstByte = getByteIndex(startIndex);
-    int byteSize = getSizeFromCount(length);
-    int offset = startIndex % 8;
-    if (offset == 0) {
+    int firstByteIndex = getByteIndex(startIndex);//byte offset of the first src byte
+    int numBytesHoldingSourceBits = getSizeFromCount(length); //src bytes to read (including start/end bytes that might not be fully copied)
+    int firstBitOffset = startIndex % 8; //Offset of first src bit within the first src byte
+    if (firstBitOffset == 0) {
       target.clear();
       // slice
       if (target.data != null) {
         target.data.release();
       }
-      target.data = data.slice(firstByte, byteSize);
+      target.data = data.slice(firstByteIndex, numBytesHoldingSourceBits);
       target.data.retain(1);
     } else {
       // Copy data
-      // When the first bit starts from the middle of a byte (offset != 0), copy data from src BitVector.
+      // When the first bit starts from the middle of a byte (firstBitOffset != 0), copy data from src BitVector.
       // Each byte in the target is composed by a part in i-th byte, another part in (i+1)-th byte.
       // The last byte copied to target is a bit tricky :
       //   1) if length requires partly byte (length % 8 !=0), copy the remaining bits only.
@@ -306,14 +306,37 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       target.clear();
       target.allocateNew(length);
       // TODO maybe do this one word at a time, rather than byte?
-      for(int i = 0; i < byteSize - 1; i++) {
-        target.data.setByte(i, (((this.data.getByte(firstByte + i) & 0xFF) >>> offset) + (this.data.getByte(firstByte + i + 1) <<  (8 - offset))));
+
+      byte byteI, byteIPlus1 = 0;
+      for(int i = 0; i < numBytesHoldingSourceBits - 1; i++) {
+        byteI = this.data.getByte(firstByteIndex + i);
+        byteIPlus1 = this.data.getByte(firstByteIndex + i + 1);
+        // Extract higher-X bits from first byte i and lower-Y bits from byte (i + 1), where X + Y = 8 bits
+        // Lower-Y  bits are the MS bits of the byte to be written (target byte) and Higher-X are the LS bits.
+        // The target bytes are assembled in accordance to little-endian ordering (byte[0] = LS bit, byte[7] = MS bit)
+        target.data.setByte(i, (((byteI & 0xFF) >>> firstBitOffset) + (byteIPlus1 <<  (8 - firstBitOffset))));
       }
+
+      //Copying the last n bits
+
+      //Copy length is not a byte-multiple
       if (length % 8 != 0) {
-        target.data.setByte(byteSize - 1, ((this.data.getByte(firstByte + byteSize - 1) & 0xFF) >>> offset));
+        // start is not byte aligned so we have to copy some bits from the last full byte read in the
+        // previous loop
+        byte lastButOneByte = byteIPlus1;
+        byte bitsFromLastButOneByte = (byte)((lastButOneByte & 0xFF) >>> firstBitOffset);
+
+        // If we have to read more bits than what we have already read, read it into lastByte otherwise set lastByte to 0.
+        // (length % 8) is num of remaining bits to be read.
+        // (8 - firstBitOffset) is the number of bits already read into lastButOneByte but not used in the previous write.
+        // We do not have to read more bits if (8 - firstBitOffset >= length % 8)
+        final int lastByte = (8 - firstBitOffset >= length % 8) ?
+                0 : this.data.getByte(firstByteIndex + numBytesHoldingSourceBits);
+        target.data.setByte(numBytesHoldingSourceBits - 1, bitsFromLastButOneByte + (lastByte << (8 - firstBitOffset)));
       } else {
-        target.data.setByte(byteSize - 1,
-            (((this.data.getByte(firstByte + byteSize - 1) & 0xFF) >>> offset) + (this.data.getByte(firstByte + byteSize) <<  (8 - offset))));
+        target.data.setByte(numBytesHoldingSourceBits - 1,
+            (((this.data.getByte(firstByteIndex + numBytesHoldingSourceBits - 1) & 0xFF) >>> firstBitOffset) +
+                     (this.data.getByte(firstByteIndex + numBytesHoldingSourceBits) <<  (8 - firstBitOffset))));
       }
     }
     target.getMutator().setValueCount(length);