You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/05/31 23:07:00 UTC

[jira] [Commented] (DRILL-3313) Eliminate redundant #load methods and unit-test loading & exporting of vectors

    [ https://issues.apache.org/jira/browse/DRILL-3313?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16497297#comment-16497297 ] 

ASF GitHub Bot commented on DRILL-3313:
---------------------------------------

ilooner closed pull request #81: DRILL-3313: Eliminate redundant #load methods and unit-test loading & exporting of vectors
URL: https://github.com/apache/drill/pull/81
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java b/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java
index cadcfd92ba..fa14fce059 100644
--- a/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java
@@ -156,21 +156,17 @@ public void zeroVector() {
   }
 
   @Override
-  public int load(int valueCount, DrillBuf buf){
+  public void load(SerializedField metadata, DrillBuf buffer) {
+    final int actualLength = metadata.getBufferLength();
+    final int valueCount = metadata.getValueCount();
+    final int expectedLength = valueCount * ${type.width};
+    assert actualLength == expectedLength : String.format("Expected to load %d bytes but actually loaded %d bytes", expectedLength, actualLength);
+
     clear();
-    int len = valueCount * ${type.width};
-    data = buf.slice(0, len);
+    data = buffer.slice(0, actualLength);
+    data.writerIndex(actualLength);
     data.retain();
-    data.writerIndex(len);
-    return len;
-  }
-
-  @Override
-  public void load(SerializedField metadata, DrillBuf buffer) {
-    assert this.field.matches(metadata) : String.format("The field %s doesn't match the provided metadata %s.", this.field, metadata);
-    int loaded = load(metadata.getValueCount(), buffer);
-    assert metadata.getBufferLength() == loaded : String.format("Expected to load %d bytes but actually loaded %d bytes", metadata.getBufferLength(), loaded);
-  }
+    }
 
   public TransferPair getTransferPair(){
     return new TransferImpl(getField());
diff --git a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
index 7fa0d551fe..7b8d3eb160 100644
--- a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
@@ -50,8 +50,10 @@
 
   private final FieldReader reader = new Nullable${minor.class}ReaderImpl(Nullable${minor.class}Vector.this);
 
-  private final UInt1Vector bits = new UInt1Vector(MaterializedField.create(field + "_bits", Types.required(MinorType.UINT1)), allocator);
+  private final MaterializedField bitsField = MaterializedField.create("$bits$", Types.required(MinorType.UINT1));
+  private final UInt1Vector bits = new UInt1Vector(bitsField, allocator);
   private final ${valuesName} values = new ${minor.class}Vector(field, allocator);
+
   private final Mutator mutator = new Mutator();
   private final Accessor accessor = new Accessor();
 
@@ -106,16 +108,14 @@ public void setInitialCapacity(int numRecords) {
     values.setInitialCapacity(numRecords);
   }
 
-  <#if type.major == "VarLen">
   @Override
-  public SerializedField getMetadata() {
-    return getMetadataBuilder()
-             .setValueCount(getAccessor().getValueCount())
-             .setVarByteLength(values.getVarByteLength())
-             .setBufferLength(getBufferSize())
-             .build();
+  public SerializedField.Builder getMetadataBuilder() {
+    return super.getMetadataBuilder()
+      .addChild(bits.getMetadata())
+      .addChild(values.getMetadata());
   }
 
+  @Override
   public void allocateNew() {
     if(!allocateNewSafe()){
       throw new OutOfMemoryRuntimeException("Failure while allocating buffer.");
@@ -143,6 +143,7 @@ public boolean allocateNewSafe() {
     return success;
   }
 
+  <#if type.major == "VarLen">
   @Override
   public void allocateNew(int totalBytes, int valueCount) {
     try {
@@ -157,26 +158,6 @@ public void allocateNew(int totalBytes, int valueCount) {
     accessor.reset();
   }
 
-  @Override
-  public int load(int dataBytes, int valueCount, DrillBuf buf){
-    clear();
-    int loaded = bits.load(valueCount, buf);
-
-    // remove bits part of buffer.
-    buf = buf.slice(loaded, buf.capacity() - loaded);
-    dataBytes -= loaded;
-    loaded += values.load(dataBytes, valueCount, buf);
-    this.mutator.lastSet = valueCount;
-    return loaded;
-  }
-
-  @Override
-  public void load(SerializedField metadata, DrillBuf buffer) {
-    assert this.field.matches(metadata) : String.format("The field %s doesn't match the provided metadata %s.", this.field, metadata);
-    int loaded = load(metadata.getBufferLength(), metadata.getValueCount(), buffer);
-    assert metadata.getBufferLength() == loaded : String.format("Expected to load %d bytes but actually loaded %d bytes", metadata.getBufferLength(), loaded);
-  }
-
   @Override
   public int getByteCapacity(){
     return values.getByteCapacity();
@@ -188,48 +169,11 @@ public int getCurrentSizeInBytes(){
   }
 
   <#else>
-
-  @Override
-  public void allocateNew() {
-    try {
-      values.allocateNew();
-      bits.allocateNew();
-    } catch(DrillRuntimeException e) {
-      clear();
-      throw e;
-    }
-    bits.zeroVector();
-    mutator.reset();
-    accessor.reset();
-  }
-
-
-  @Override
-  public boolean allocateNewSafe() {
-    /* Boolean to keep track if all the memory allocations were successful
-     * Used in the case of composite vectors when we need to allocate multiple
-     * buffers for multiple vectors. If one of the allocations failed we need to
-     * clear all the memory that we allocated
-     */
-    boolean success = false;
-    try {
-      success = values.allocateNewSafe() && bits.allocateNewSafe();
-    } finally {
-      if (!success) {
-        clear();
-      }
-    }
-    bits.zeroVector();
-    mutator.reset();
-    accessor.reset();
-    return success;
-  }
-
   @Override
   public void allocateNew(int valueCount) {
     try {
       values.allocateNew(valueCount);
-      bits.allocateNew(valueCount);
+      bits.allocateNew(valueCount+1);
     } catch(OutOfMemoryRuntimeException e) {
       clear();
       throw e;
@@ -239,37 +183,30 @@ public void allocateNew(int valueCount) {
     accessor.reset();
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  @Override
   public void zeroVector() {
-    this.values.zeroVector();
-    this.bits.zeroVector();
+    bits.zeroVector();
+    values.zeroVector();
   }
+  </#if>
 
-  @Override
-  public int load(int valueCount, DrillBuf buf){
-    clear();
-    int loaded = bits.load(valueCount, buf);
-
-    // remove bits part of buffer.
-    buf = buf.slice(loaded, buf.capacity() - loaded);
-    loaded += values.load(valueCount, buf);
-    return loaded;
-  }
 
   @Override
   public void load(SerializedField metadata, DrillBuf buffer) {
-    assert this.field.matches(metadata);
-    int loaded = load(metadata.getValueCount(), buffer);
-    assert metadata.getBufferLength() == loaded;
-  }
+    clear();
+    final SerializedField bitsField = metadata.getChild(0);
+    bits.load(bitsField, buffer);
 
-  </#if>
+    final int capacity = buffer.capacity();
+    final int bitsLength = bitsField.getBufferLength();
+    final SerializedField valuesField = metadata.getChild(1);
+    values.load(valuesField, buffer.slice(bitsLength, capacity - bitsLength));
+  }
 
   public TransferPair getTransferPair(){
     return new TransferImpl(getField());
   }
+
   public TransferPair getTransferPair(FieldReference ref){
     return new TransferImpl(getField().withPath(ref));
   }
@@ -278,7 +215,6 @@ public TransferPair makeTransferPair(ValueVector to) {
     return new TransferImpl((Nullable${minor.class}Vector) to);
   }
 
-
   public void transferTo(Nullable${minor.class}Vector target){
     bits.transferTo(target.bits);
     values.transferTo(target.values);
@@ -296,7 +232,7 @@ public void splitAndTransferTo(int startIndex, int length, Nullable${minor.class
     </#if>
   }
 
-  private class TransferImpl implements TransferPair{
+  private class TransferImpl implements TransferPair {
     Nullable${minor.class}Vector to;
 
     public TransferImpl(MaterializedField field){
diff --git a/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java b/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
index fc08ac664c..afeafaad03 100644
--- a/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
@@ -57,10 +57,9 @@
   private static final int INITIAL_BYTE_COUNT = 4096 * DEFAULT_RECORD_BYTE_COUNT;
   private static final int MIN_BYTE_COUNT = 4096;
 
-  public final static String OFFSETS_VECTOR_NAME = "offsets";
-  private final static MaterializedField offsetsField =
-    MaterializedField.create(OFFSETS_VECTOR_NAME, Types.required(MinorType.UINT4));
-  private final UInt${type.width}Vector offsetVector;
+  public final static String OFFSETS_VECTOR_NAME = "$offsets$";
+  private final MaterializedField offsetsField = MaterializedField.create(OFFSETS_VECTOR_NAME, Types.required(MinorType.UINT4));
+  private final UInt${type.width}Vector offsetVector = new UInt${type.width}Vector(offsetsField, allocator);
   private final FieldReader reader = new ${minor.class}ReaderImpl(${minor.class}Vector.this);
 
   private final Accessor accessor;
@@ -74,7 +73,6 @@
 
   public ${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
-    this.offsetVector = new UInt${type.width}Vector(offsetsField, allocator);
     this.oAccessor = offsetVector.getAccessor();
     this.accessor = new Accessor();
     this.mutator = new Mutator();
@@ -95,7 +93,7 @@ int getSizeFromCount(int valueCount) {
   }
 
   public int getValueCapacity(){
-    return offsetVector.getValueCapacity() - 1;
+    return Math.max(offsetVector.getValueCapacity() - 1, 0);
   }
 
   public int getByteCapacity(){
@@ -119,29 +117,22 @@ public int getVarByteLength(){
   @Override
   public SerializedField getMetadata() {
     return getMetadataBuilder() //
+             .addChild(offsetVector.getMetadata())
              .setValueCount(getAccessor().getValueCount()) //
-             .setVarByteLength(getVarByteLength()) //
+//             .setVarByteLength(getVarByteLength()) //
              .setBufferLength(getBufferSize()) //
              .build();
   }
 
-  public int load(int dataBytes, int valueCount, DrillBuf buf){
-    if(valueCount == 0){
-      allocateNew(0,0);
-      return 0;
-    }
-    clear();
-    int loaded = offsetVector.load(valueCount+1, buf);
-    data = buf.slice(loaded, dataBytes - loaded);
-    data.retain();
-    return  dataBytes;
-  }
-
   @Override
   public void load(SerializedField metadata, DrillBuf buffer) {
-    assert this.field.matches(metadata) : String.format("The field %s doesn't match the provided metadata %s.", this.field, metadata);
-    int loaded = load(metadata.getBufferLength(), metadata.getValueCount(), buffer);
-    assert metadata.getBufferLength() == loaded : String.format("Expected to load %d bytes but actually loaded %d bytes", metadata.getBufferLength(), loaded);
+    final SerializedField offsetField = metadata.getChild(0);
+    offsetVector.load(offsetField, buffer);
+
+    final int capacity = buffer.capacity();
+    final int offsetsLength = offsetField.getBufferLength();
+    data = buffer.slice(offsetsLength, capacity - offsetsLength);
+    data.retain();
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
index 054ef82612..65e88e601a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -152,16 +152,6 @@ public void zeroVector() {
     data.setZero(0, data.capacity());
   }
 
-  @Override
-  public int load(int valueCount, DrillBuf buf) {
-    clear();
-    this.valueCount = valueCount;
-    int len = getSizeFromCount(valueCount);
-    data = buf.slice(0, len);
-    data.retain();
-    return len;
-  }
-
   public void copyFrom(int inIndex, int outIndex, BitVector from) {
     this.mutator.set(outIndex, from.accessor.get(inIndex));
   }
@@ -177,9 +167,16 @@ public boolean copyFromSafe(int inIndex, int outIndex, BitVector from) {
 
   @Override
   public void load(SerializedField metadata, DrillBuf buffer) {
-    assert this.field.matches(metadata);
-    int loaded = load(metadata.getValueCount(), buffer);
-    assert metadata.getBufferLength() == loaded;
+    assert field.matches(metadata);
+    final int valueCount = metadata.getValueCount();
+    final int expectedLength = getSizeFromCount(valueCount);
+    final int actualLength = metadata.getBufferLength();
+    assert expectedLength == actualLength: "expected and actual buffer sizes do not match";
+
+    clear();
+    data = buffer.slice(0, actualLength);
+    data.retain();
+    this.valueCount = valueCount;
   }
 
   public Mutator getMutator() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
index b546b9efa3..529c05483b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.vector;
 
-import io.netty.buffer.DrillBuf;
 
 public interface FixedWidthVector extends ValueVector{
 
@@ -26,21 +25,8 @@
    *
    * @param valueCount   Number of values in the vector.
    */
-  public void allocateNew(int valueCount);
-
-  /**
-   * Load the records in the provided buffer based on the given number of values.
-   * @param valueCount Number of values the buffer contains.
-   * @param buf Incoming buffer.
-   * @return The number of bytes of the buffer that were consumed.
-   */
-  public int load(int valueCount, DrillBuf buf);
+  void allocateNew(int valueCount);
 
+  void zeroVector();
 
-  public abstract Mutator getMutator();
-
-  /**
-   * Zero out the underlying buffer backing this vector.
-   */
-  public void zeroVector();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NullableVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NullableVector.java
index 92f60d66ee..8091c4c56a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NullableVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NullableVector.java
@@ -19,5 +19,5 @@
 
 public interface NullableVector extends ValueVector{
 
-  public ValueVector getValuesVector();
+  ValueVector getValuesVector();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
index 10ddcc9cee..ee9c0396e5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
@@ -27,34 +27,25 @@
    * @param totalBytes   Desired size of the underlying data buffer.
    * @param valueCount   Number of values in the vector.
    */
-  public void allocateNew(int totalBytes, int valueCount);
+  void allocateNew(int totalBytes, int valueCount);
 
   /**
    * Provide the maximum amount of variable width bytes that can be stored int his vector.
    * @return
    */
-  public int getByteCapacity();
+  int getByteCapacity();
 
-  /**
-   * Load the records in the provided buffer based on the given number of values.
-   * @param dataBytes   The number of bytes associated with the data array.
-   * @param valueCount Number of values the buffer contains.
-   * @param buf Incoming buffer.
-   * @return The number of bytes of the buffer that were consumed.
-   */
-  public int load(int dataBytes, int valueCount, DrillBuf buf);
-
-  public abstract VariableWidthMutator getMutator();
+  VariableWidthMutator getMutator();
 
-  public abstract VariableWidthAccessor getAccessor();
+  VariableWidthAccessor getAccessor();
 
-  public interface VariableWidthAccessor extends Accessor {
-    public int getValueLength(int index);
+  interface VariableWidthAccessor extends Accessor {
+    int getValueLength(int index);
   }
 
-  public int getCurrentSizeInBytes();
+  int getCurrentSizeInBytes();
 
-  public interface VariableWidthMutator extends Mutator {
-    public void setValueLengthSafe(int index, int length);
+  interface VariableWidthMutator extends Mutator {
+    void setValueLengthSafe(int index, int length);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
index 9bf6d8572d..fcec2ab06b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
@@ -100,7 +100,7 @@ public void setInitialCapacity(int numRecords) {
 
   @Override
   public int getValueCapacity() {
-    final int offsetValueCapacity = offsets.getValueCapacity() - 1;
+    final int offsetValueCapacity = Math.max(offsets.getValueCapacity() - 1, 0);
     if (vector == DEFAULT_DATA_VECTOR) {
       return offsetValueCapacity;
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
index 1e30ea2117..a06ce66051 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
@@ -33,6 +33,7 @@
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.expr.holders.ComplexHolder;
 import org.apache.drill.exec.memory.BufferAllocator;
@@ -52,7 +53,7 @@
 public class MapVector extends AbstractMapVector {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapVector.class);
 
-  public final static MajorType TYPE = MajorType.newBuilder().setMinorType(MinorType.MAP).setMode(DataMode.REQUIRED).build();
+  public final static MajorType TYPE = Types.required(MinorType.MAP);
 
   private final SingleMapReaderImpl reader = new SingleMapReaderImpl(MapVector.this);
   private final Accessor accessor = new Accessor();
@@ -224,7 +225,7 @@ public void splitAndTransfer(int startIndex, int length) {
   @Override
   public int getValueCapacity() {
     if (size() == 0) {
-      return Integer.MAX_VALUE;
+      return 0;
     }
 
     final Ordering<ValueVector> natural = new Ordering<ValueVector>() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
index 644e5db67e..40d0be4c96 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
@@ -415,7 +415,7 @@ public void copyFromSafe(int fromIndex, int thisIndex, RepeatedMapVector from) {
 
   @Override
   public int getValueCapacity() {
-    return offsets.getValueCapacity()-1;
+    return Math.max(offsets.getValueCapacity() - 1, 0);
   }
 
   @Override
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 4eeb8f282e..339dbe8b0d 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
@@ -23,8 +23,13 @@
 
 import java.nio.charset.Charset;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.exception.OversizedAllocationException;
 import org.apache.drill.exec.expr.TypeHelper;
@@ -36,9 +41,11 @@
 import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
 import org.apache.drill.exec.expr.holders.RepeatedFloat4Holder;
 import org.apache.drill.exec.expr.holders.RepeatedVarBinaryHolder;
+import org.apache.drill.exec.expr.holders.UInt1Holder;
 import org.apache.drill.exec.expr.holders.UInt4Holder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.BitVector;
@@ -480,4 +487,116 @@ public void testVVInitialCapacity() throws Exception {
       AutoCloseables.close(valueVectors);
     }
   }
+
+  protected interface VectorVerifier {
+    void verify(ValueVector vector) throws Exception;
+  }
+
+  protected static class ChildVerifier implements VectorVerifier {
+    public final TypeProtos.MajorType[] types;
+
+    public ChildVerifier(TypeProtos.MajorType... childTypes) {
+      this.types = Preconditions.checkNotNull(childTypes);
+    }
+
+    @Override
+    public void verify(ValueVector vector) throws Exception {
+      final String hint = String.format("%s failed the test case", vector.getClass().getSimpleName());
+
+      final UserBitShared.SerializedField metadata = vector.getMetadata();
+      final int actual = metadata.getChildCount();
+      assertEquals(hint, types.length, actual);
+
+      for (int i = 0; i < types.length; i++) {
+        final UserBitShared.SerializedField child = metadata.getChild(i);
+
+        assertEquals(hint, types[i], child.getMajorType());
+      }
+    }
+  }
+
+  /**
+   * Convenience method that allows running tests on various {@link ValueVector vector} instances.
+   *
+   * @param test test function to execute
+   */
+  private void testVectors(VectorVerifier test) throws Exception {
+    final MaterializedField[] fields = {
+        MaterializedField.create(EMPTY_SCHEMA_PATH, UInt1Holder.TYPE),
+        MaterializedField.create(EMPTY_SCHEMA_PATH, BitHolder.TYPE),
+        MaterializedField.create(EMPTY_SCHEMA_PATH, VarCharHolder.TYPE),
+        MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVarCharHolder.TYPE),
+        MaterializedField.create(EMPTY_SCHEMA_PATH, RepeatedListVector.TYPE),
+        MaterializedField.create(EMPTY_SCHEMA_PATH, MapVector.TYPE),
+        MaterializedField.create(EMPTY_SCHEMA_PATH, RepeatedMapVector.TYPE)
+    };
+
+    final ValueVector[] vectors = {
+        new UInt4Vector(fields[0], allocator),
+        new BitVector(fields[1], allocator),
+        new VarCharVector(fields[2], allocator),
+        new NullableVarCharVector(fields[3], allocator),
+        new RepeatedListVector(fields[4], allocator, null),
+        new MapVector(fields[5], allocator, null),
+        new RepeatedMapVector(fields[6], allocator, null)
+    };
+
+    try {
+      for (final ValueVector vector : vectors) {
+        test.verify(vector);
+      }
+    } finally {
+      AutoCloseables.close(vectors);
+    }
+  }
+
+  @Test
+  public void testVectorMetadataIsAccurate() throws Exception {
+    final VectorVerifier noChild = new ChildVerifier();
+    final VectorVerifier offsetChild = new ChildVerifier(UInt4Holder.TYPE);
+
+    final ImmutableMap.Builder<Class, VectorVerifier> builder = ImmutableMap.builder();
+    builder.put(UInt4Vector.class, noChild);
+    builder.put(BitVector.class, noChild);
+    builder.put(VarCharVector.class, offsetChild);
+    builder.put(NullableVarCharVector.class, new ChildVerifier(UInt1Holder.TYPE, Types.optional(TypeProtos.MinorType.VARCHAR)));
+    builder.put(RepeatedListVector.class, new ChildVerifier(UInt4Holder.TYPE, Types.LATE_BIND_TYPE));
+    builder.put(MapVector.class, noChild);
+    builder.put(RepeatedMapVector.class, offsetChild);
+    final ImmutableMap<Class, VectorVerifier> children = builder.build();
+
+    testVectors(new VectorVerifier() {
+
+      @Override
+      public void verify(ValueVector vector) throws Exception {
+
+        final Class klazz = vector.getClass();
+        final VectorVerifier verifier = children.get(klazz);
+        verifier.verify(vector);
+      }
+    });
+  }
+
+  @Test
+  public void testVectorCanLoadEmptyBuffer() throws Exception {
+    final DrillBuf empty = allocator.getEmpty();
+
+    testVectors(new VectorVerifier() {
+
+      @Override
+      public void verify(ValueVector vector) {
+        final String hint = String.format("%s failed the test case", vector.getClass().getSimpleName());
+        final UserBitShared.SerializedField metadata = vector.getMetadata();
+        assertEquals(hint, 0, metadata.getBufferLength());
+        assertEquals(hint, 0, metadata.getValueCount());
+
+        vector.load(metadata, empty);
+
+        assertEquals(hint, 0, vector.getValueCapacity());
+        assertEquals(hint, 0, vector.getAccessor().getValueCount());
+
+        vector.clear();
+      }
+    });
+  }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Eliminate redundant #load methods and unit-test loading & exporting of vectors
> ------------------------------------------------------------------------------
>
>                 Key: DRILL-3313
>                 URL: https://issues.apache.org/jira/browse/DRILL-3313
>             Project: Apache Drill
>          Issue Type: Sub-task
>          Components: Execution - Data Types
>    Affects Versions: 1.0.0
>            Reporter: Hanifi Gunes
>            Assignee: Hanifi Gunes
>            Priority: Major
>             Fix For: 1.3.0
>
>
> Vectors have multiple #load methods that are used to populate data from raw buffers. It is relatively tough to reason, maintain and unit-test loading and exporting of data since there is many redundant code around load methods. This issue proposes to have single #load method conforming to VV#load(def, buffer) signature eliminating all other #load overrides.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)