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/03/26 11:35:04 UTC

[09/13] drill git commit: DRILL-6262: IndexOutOfBoundException in RecordBatchSize for empty variableWidthVector

DRILL-6262: IndexOutOfBoundException in RecordBatchSize for empty variableWidthVector

closes #1175


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

Branch: refs/heads/master
Commit: 051a96dae42c70e633ba3f6af9817836544e075b
Parents: 8663e8a
Author: Sorabh Hamirwasia <sh...@maprtech.com>
Authored: Fri Mar 16 16:57:12 2018 -0700
Committer: Vitalii Diravka <vi...@gmail.com>
Committed: Mon Mar 26 13:02:56 2018 +0300

----------------------------------------------------------------------
 .../drill/exec/record/RecordBatchSizer.java     |  9 ++--
 .../drill/exec/record/TestRecordBatchSizer.java | 43 ++++++++++++++++++--
 .../codegen/templates/RepeatedValueVectors.java |  2 +-
 .../templates/VariableLengthVectors.java        |  8 +++-
 4 files changed, 50 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/051a96da/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
index bfe0ef1..1586ccc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
@@ -321,10 +321,8 @@ public class RecordBatchSizer {
 
           // Calculate pure data size.
           if (isVariableWidth) {
-            UInt4Vector offsetVector = ((RepeatedValueVector) v).getOffsetVector();
-            int innerValueCount = offsetVector.getAccessor().get(valueCount);
             VariableWidthVector dataVector = ((VariableWidthVector) ((RepeatedValueVector) v).getDataVector());
-            totalDataSize = dataVector.getOffsetVector().getAccessor().get(innerValueCount);
+            totalDataSize = dataVector.getCurrentSizeInBytes();
           } else {
             ValueVector dataVector = ((RepeatedValueVector) v).getDataVector();
             totalDataSize = dataVector.getPayloadByteCount(elementCount);
@@ -343,7 +341,7 @@ public class RecordBatchSizer {
           // Calculate pure data size.
           if (isVariableWidth) {
             VariableWidthVector variableWidthVector = ((VariableWidthVector) ((NullableVector) v).getValuesVector());
-            totalDataSize = variableWidthVector.getOffsetVector().getAccessor().get(valueCount);
+            totalDataSize = variableWidthVector.getCurrentSizeInBytes();
           } else {
             // Another special case.
             if (v instanceof UntypedNullVector) {
@@ -362,8 +360,7 @@ public class RecordBatchSizer {
 
           // Calculate pure data size.
           if (isVariableWidth) {
-            UInt4Vector  offsetVector = ((VariableWidthVector)v).getOffsetVector();
-            totalDataSize = offsetVector.getAccessor().get(valueCount);
+            totalDataSize = ((VariableWidthVector) v).getCurrentSizeInBytes();
           } else {
             totalDataSize = v.getPayloadByteCount(valueCount);
           }

http://git-wip-us.apache.org/repos/asf/drill/blob/051a96da/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordBatchSizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordBatchSizer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordBatchSizer.java
index f32dba3..ccb9c19 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordBatchSizer.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordBatchSizer.java
@@ -17,9 +17,6 @@
  */
 package org.apache.drill.exec.record;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.record.RecordBatchSizer.ColumnSize;
 import org.apache.drill.exec.vector.NullableVector;
@@ -36,6 +33,10 @@ import org.apache.drill.test.rowSet.RowSetBuilder;
 import org.apache.drill.test.rowSet.schema.SchemaBuilder;
 import org.junit.Test;
 
+import static junit.framework.TestCase.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
 public class TestRecordBatchSizer extends SubOperatorTest {
   private final int testRowCount = 1000;
   private final int testRowCountPowerTwo = 2048;
@@ -804,4 +805,38 @@ public class TestRecordBatchSizer extends SubOperatorTest {
 
   }
 
-}
\ No newline at end of file
+  /**
+   * Test to verify that record batch sizer handles the actual empty vectors correctly. RowSetBuilder by default
+   * allocates Drillbuf of 10bytes for each vector type which makes their capacity >0 and not ==0 which will be in
+   * case of empty vectors.
+   */
+  @Test
+  public void testEmptyVariableWidthVector() {
+    final BatchSchema schema = new SchemaBuilder()
+      .add("key", MinorType.INT)
+      .add("value", MinorType.VARCHAR)
+      .build();
+
+    final RowSetBuilder builder = fixture.rowSetBuilder(schema);
+    final RowSet rows = builder.build();
+
+    // Release the initial bytes allocated by RowSetBuilder
+    VectorAccessibleUtilities.clear(rows.container());
+
+    try {
+      // Create RecordBatchSizer for this empty container and it should not fail
+      final RecordBatchSizer sizer = new RecordBatchSizer(rows.container());
+      int totalDataSize = 0;
+
+      for (ColumnSize colSize : sizer.columns().values()) {
+        totalDataSize += colSize.getTotalDataSize();
+      }
+      // Verify that the totalDataSize for all the columns is zero
+      assertEquals(0, totalDataSize);
+    } catch (Exception ex) {
+      fail();
+    } finally {
+      rows.clear();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/051a96da/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java b/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
index c20ee89..418898b 100644
--- a/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
@@ -228,7 +228,7 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
   @Override
   protected SerializedField.Builder getMetadataBuilder() {
     return super.getMetadataBuilder()
-            .setVarByteLength(values.getVarByteLength());
+            .setVarByteLength(values.getCurrentSizeInBytes());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/051a96da/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
index 3dec3f8..516eb52 100644
--- a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
@@ -111,9 +111,15 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     return data.capacity();
   }
 
+  /**
+  * Return the number of bytes contained in the current var len byte vector.
+  * TODO: Remove getVarByteLength with it's implementation after all client's are moved to using getCurrentSizeInBytes.
+  * It's kept as is to preserve backward compatibility
+  * @return
+  */
   @Override
   public int getCurrentSizeInBytes() {
-    return offsetVector.getAccessor().get(getAccessor().getValueCount());
+    return getVarByteLength();
   }
 
   /**