You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by me...@apache.org on 2015/04/16 23:15:27 UTC

[1/3] drill git commit: DRILL-2356: Fix round function for exact input types

Repository: drill
Updated Branches:
  refs/heads/master cb47df0c8 -> 64e3ec52b


DRILL-2356: Fix round function for exact input types


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

Branch: refs/heads/master
Commit: a7e959040dac7562aed6f86a62e21a2d192d67b0
Parents: cb47df0
Author: Mehant Baid <me...@gmail.com>
Authored: Tue Apr 14 17:59:34 2015 -0700
Committer: Mehant Baid <me...@gmail.com>
Committed: Thu Apr 16 10:48:36 2015 -0700

----------------------------------------------------------------------
 .../src/main/codegen/data/MathFunc.tdd          |  13 --
 .../exec/expr/fn/impl/conv/RoundFunctions.java  | 193 +++++++++++++++++++
 .../org/apache/drill/TestFunctionsQuery.java    |  18 ++
 3 files changed, 211 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a7e95904/exec/java-exec/src/main/codegen/data/MathFunc.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/MathFunc.tdd b/exec/java-exec/src/main/codegen/data/MathFunc.tdd
index 154836c..c8bd95c 100644
--- a/exec/java-exec/src/main/codegen/data/MathFunc.tdd
+++ b/exec/java-exec/src/main/codegen/data/MathFunc.tdd
@@ -56,19 +56,6 @@ unaryMathFunctions : [
       {input: "UInt8", outputType: "UInt8", castType: "long"}
      ]
    	},
-  	{className: "Round", funcName: "round", javaFunc : "java.lang.Math.round", types: [
-      {input: "Int",  outputType: "Int", castType: "int"},
-      {input: "BigInt",  outputType: "BigInt", castType: "long"},
-      {input: "Float4", outputType: "Float4", castType: "float"},
-      {input: "Float8", outputType: "Float8", castType: "double"},
-      {input: "SmallInt",  outputType: "SmallInt", castType: "short"},
-      {input: "TinyInt",  outputType: "TinyInt", castType: "byte"},
-      {input: "UInt1", outputType: "UInt1", castType: "byte"},
-      {input: "UInt2", outputType: "UInt2", castType: "char"},
-      {input: "UInt4", outputType: "UInt4", castType: "int"},
-      {input: "UInt8", outputType: "UInt8", castType: "long"}
-     ]
-   	},
   	{className: "Sqrt", funcName: "sqrt", javaFunc : "java.lang.Math.sqrt", types: [
       {input: "Int",  outputType: "Int", castType: "int"},
       {input: "BigInt",  outputType: "BigInt", castType: "long"},

http://git-wip-us.apache.org/repos/asf/drill/blob/a7e95904/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/RoundFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/RoundFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/RoundFunctions.java
new file mode 100644
index 0000000..db39d15
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/RoundFunctions.java
@@ -0,0 +1,193 @@
+/*******************************************************************************
+
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.expr.fn.impl.conv;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.SmallIntHolder;
+import org.apache.drill.exec.expr.holders.TinyIntHolder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.Float4Holder;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.UInt1Holder;
+import org.apache.drill.exec.expr.holders.UInt2Holder;
+import org.apache.drill.exec.expr.holders.UInt4Holder;
+import org.apache.drill.exec.expr.holders.UInt8Holder;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+
+
+public class RoundFunctions {
+
+  /*
+   * Following are round functions with no parameter. Per the SQL standard we simply return the same output
+   * type as the input type for exact inputs (int, bigint etc) and inexact types (float, double).
+   *
+   * TODO: Need to incorporate round function which accepts two parameters here.
+   */
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class RoundInt implements DrillSimpleFunc {
+
+    @Param  IntHolder in;
+    @Output IntHolder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RoundBigInt implements DrillSimpleFunc {
+
+    @Param BigIntHolder in;
+    @Output BigIntHolder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RoundSmallInt implements DrillSimpleFunc {
+
+    @Param SmallIntHolder in;
+    @Output SmallIntHolder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+
+
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RoundTinyInt implements DrillSimpleFunc {
+
+    @Param TinyIntHolder in;
+    @Output TinyIntHolder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+
+
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RoundUInt1 implements DrillSimpleFunc {
+
+    @Param UInt1Holder in;
+    @Output UInt1Holder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+
+
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RoundUInt2 implements DrillSimpleFunc {
+
+    @Param UInt2Holder in;
+    @Output UInt2Holder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+
+
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RoundUInt4 implements DrillSimpleFunc {
+
+    @Param UInt4Holder in;
+    @Output UInt4Holder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+
+
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RoundUInt8 implements DrillSimpleFunc {
+
+    @Param UInt8Holder in;
+    @Output UInt8Holder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RoundFloat4 implements DrillSimpleFunc {
+
+    @Param Float4Holder in;
+    @Output Float4Holder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      java.math.BigDecimal input = java.math.BigDecimal.valueOf(in.value);
+      out.value = input.setScale(0, java.math.RoundingMode.HALF_UP).floatValue();
+    }
+  }
+
+  @FunctionTemplate(name = "round", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RoundFloat8 implements DrillSimpleFunc {
+
+    @Param Float8Holder in;
+    @Output Float8Holder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      java.math.BigDecimal input = java.math.BigDecimal.valueOf(in.value);
+      out.value = input.setScale(0, java.math.RoundingMode.HALF_UP).doubleValue();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/a7e95904/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
index 9a483a1..4b9001f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
@@ -472,6 +472,24 @@ public class TestFunctionsQuery extends BaseTestQuery {
   }
 
   @Test
+  public void testRoundWithOneParam() throws Exception {
+    String query = "select " +
+        "round(8124674407369523212) round_bigint," +
+        "round(9999999) round_int, " +
+        "round(cast('23.45' as float)) round_float_1, " +
+        "round(cast('23.55' as float)) round_float_2, " +
+        "round(8124674407369.2345) round_double_1, " +
+        "round(8124674407369.589) round_double_2 " +
+        " from cp.`tpch/region.parquet` limit 1";
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("round_bigint", "round_int", "round_float_1", "round_float_2", "round_double_1", "round_double_2")
+        .baselineValues(8124674407369523212l, 9999999, 23.0f, 24.0f, 8124674407369.0d, 8124674407370.0d)
+        .go();
+  }
+
+  @Test
   public void testToCharFunction() throws Exception {
     String query = "SELECT " +
         "to_char(1234.5567, '#,###.##') as FLOAT8_1, " +


[3/3] drill git commit: DRILL-2611: value vectors should report valid value count

Posted by me...@apache.org.
DRILL-2611: value vectors should report valid value count


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

Branch: refs/heads/master
Commit: 64e3ec52b93e9331aa5179e040eca19afece8317
Parents: caf779d
Author: Hanifi Gunes <hg...@maprtech.com>
Authored: Mon Apr 13 15:15:34 2015 -0700
Committer: Mehant Baid <me...@gmail.com>
Committed: Thu Apr 16 10:53:34 2015 -0700

----------------------------------------------------------------------
 .../main/codegen/templates/ComplexWriters.java  |  16 +--
 .../codegen/templates/FixedValueVectors.java    | 103 ++++++++-----------
 .../codegen/templates/NullableValueVectors.java |  60 +----------
 .../codegen/templates/RepeatedValueVectors.java |  17 +--
 .../templates/VariableLengthVectors.java        |  18 ++--
 .../drill/exec/expr/GetSetVectorHelper.java     |  10 +-
 .../parquet/columnreaders/ColumnReader.java     |   3 +-
 .../columnreaders/FixedByteAlignedReader.java   |   4 +-
 .../columnreaders/FixedWidthRepeatedReader.java |   4 +-
 .../columnreaders/NullableColumnReader.java     |   7 +-
 .../NullableFixedByteAlignedReaders.java        |   4 +-
 .../columnreaders/VarLengthColumnReaders.java   |  24 ++---
 .../drill/exec/vector/BaseDataValueVector.java  |  36 +------
 .../drill/exec/vector/BaseValueVector.java      |   9 +-
 .../org/apache/drill/exec/vector/BitVector.java |   8 +-
 15 files changed, 107 insertions(+), 216 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java b/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
index d40dfd2..49c75d1 100644
--- a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
+++ b/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
@@ -77,18 +77,18 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
 
   public void write(${minor.class?cap_first}Holder h){
     mutator.addSafe(idx(), h);
-    vector.setCurrentValueCount(idx());
+    vector.getMutator().setValueCount(idx());
   }
   
   public void write(Nullable${minor.class?cap_first}Holder h){
     mutator.addSafe(idx(), h);
-    vector.setCurrentValueCount(idx());
+    vector.getMutator().setValueCount(idx());
   }
 
   <#if !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>){
     mutator.addSafe(idx(), <#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-    vector.setCurrentValueCount(idx());
+    vector.getMutator().setValueCount(idx());
   }
   </#if>
   
@@ -102,24 +102,24 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
   
   public void write(${minor.class}Holder h){
     mutator.setSafe(idx(), h);
-    vector.setCurrentValueCount(idx());
+    vector.getMutator().setValueCount(idx());
   }
   
   public void write(Nullable${minor.class}Holder h){
     mutator.setSafe(idx(), h);
-    vector.setCurrentValueCount(idx());
+    vector.getMutator().setValueCount(idx());
   }
-  
+
   <#if !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>){
     mutator.setSafe(idx(), <#if mode == "Nullable">1, </#if><#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-    vector.setCurrentValueCount(idx());
+    vector.getMutator().setValueCount(idx());
   }
 
   <#if mode == "Nullable">
   public void writeNull(){
     mutator.setNull(idx());
-    vector.setCurrentValueCount(idx());
+    vector.getMutator().setValueCount(idx());
   }
   </#if>
   </#if>

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java b/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java
index e9ec220..6a924b7 100644
--- a/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java
@@ -67,7 +67,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   public Accessor getAccessor(){
     return accessor;
   }
-  
+
   public Mutator getMutator(){
     return mutator;
   }
@@ -82,7 +82,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       throw new OutOfMemoryRuntimeException("Failure while allocating buffer.");
     }
   }
-  
+
   public boolean allocateNewSafe() {
     clear();
     if (allocationMonitor > 10) {
@@ -118,48 +118,32 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     DrillBuf newBuf = allocator.buffer(allocationValueCount * ${type.width});
     newBuf.setBytes(0, data, 0, data.capacity());
     newBuf.setZero(newBuf.capacity() / 2, newBuf.capacity() / 2);
+    newBuf.writerIndex(data.writerIndex());
     data.release();
     data = newBuf;
   }
 
-  /**
-   * {@inheritDoc}
-   */
   public void zeroVector() {
     data.setZero(0, data.capacity());
   }
 
   @Override
-  public SerializedField getMetadata() {
-    return getMetadataBuilder()
-             .setValueCount(valueCount)
-             .setBufferLength(getBufferSize())
-             .build();
-  }
-
-  public int getBufferSize() {
-    if(valueCount == 0) return 0;
-    return valueCount * ${type.width};
-  }
-
-  @Override
   public int load(int valueCount, DrillBuf buf){
     clear();
-    this.valueCount = valueCount;
     int len = valueCount * ${type.width};
     data = buf.slice(0, len);
     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());
   }
@@ -176,7 +160,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     target.data = data;
     target.data.retain();
     target.data.writerIndex(data.writerIndex());
-    target.valueCount = valueCount;
     clear();
   }
 
@@ -184,15 +167,14 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     int currentWriterIndex = data.writerIndex();
     int startPoint = startIndex * ${type.width};
     int sliceLength = length * ${type.width};
-    target.valueCount = length;
     target.data = this.data.slice(startPoint, sliceLength);
     target.data.writerIndex(sliceLength);
     target.data.retain();
   }
-  
+
   private class TransferImpl implements TransferPair{
     ${minor.class}Vector to;
-    
+
     public TransferImpl(MaterializedField field){
       this.to = new ${minor.class}Vector(field, allocator);
     }
@@ -200,11 +182,11 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public TransferImpl(${minor.class}Vector to) {
       this.to = to;
     }
-    
+
     public ${minor.class}Vector getTo(){
       return to;
     }
-    
+
     public void transfer(){
       transferTo(to);
     }
@@ -212,23 +194,23 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public void splitAndTransfer(int startIndex, int length) {
       splitAndTransferTo(startIndex, length, to);
     }
-    
+
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
       to.copyFromSafe(fromIndex, toIndex, ${minor.class}Vector.this);
     }
   }
-  
+
   public void copyFrom(int fromIndex, int thisIndex, ${minor.class}Vector from){
     <#if (type.width > 8)>
     from.data.getBytes(fromIndex * ${type.width}, data, thisIndex * ${type.width}, ${type.width});
     <#else> <#-- type.width <= 8 -->
-    data.set${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width}, 
+    data.set${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width},
         from.data.get${(minor.javaType!type.javaType)?cap_first}(fromIndex * ${type.width})
     );
     </#if> <#-- type.width -->
   }
-  
+
   public void copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from){
     while(thisIndex >= getValueCapacity()) {
         reAlloc();
@@ -250,13 +232,13 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   public final class Accessor extends BaseDataValueVector.BaseAccessor {
 
     public int getValueCount() {
-      return valueCount;
+      return data.writerIndex() / ${type.width};
     }
-    
+
     public boolean isNull(int index){
       return false;
     }
-    
+
     <#if (type.width > 8)>
 
     public ${minor.javaType!type.javaType} get(int index) {
@@ -289,7 +271,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       Period p = new Period();
       return p.plusMonths(months).plusDays(days).plusMillis(millis);
     }
-    
+
     public StringBuilder getAsStringBuilder(int index) {
 
       int offsetIndex = index * ${type.width};
@@ -349,7 +331,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       return p.plusDays(days).plusMillis(millis);
     }
 
-    
+
     public StringBuilder getAsStringBuilder(int index) {
       int offsetIndex = index * ${type.width};
 
@@ -425,12 +407,12 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public ${minor.javaType!type.javaType} get(int index) {
       return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
     }
-    
+
     <#if type.width == 4>
     public long getTwoAsLong(int index) {
       return data.getLong(index * ${type.width});
     }
-    
+
     </#if>
 
     <#if minor.class == "Date">
@@ -439,14 +421,14 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
         date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
         return date;
     }
-    
+
     <#elseif minor.class == "TimeStamp">
     public ${friendlyType} getObject(int index) {
         org.joda.time.DateTime date = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
         date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
         return date;
     }
-    
+
     <#elseif minor.class == "IntervalYear">
     public ${friendlyType} getObject(int index) {
 
@@ -472,7 +454,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
              append(years).append(yearString).
              append(months).append(monthString));
     }
-    
+
     <#elseif minor.class == "Time">
     @Override
     public DateTime getObject(int index) {
@@ -500,7 +482,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       return get(index);
     }
     </#if>
-    
+
     public void get(int index, ${minor.class}Holder holder){
       <#if minor.class.startsWith("Decimal")>
       holder.scale = getField().getScale();
@@ -509,7 +491,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
       holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
     }
-    
+
     public void get(int index, Nullable${minor.class}Holder holder){
       holder.isSet = 1;
       holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
@@ -518,7 +500,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
    </#if> <#-- type.width -->
  }
- 
+
  /**
   * ${minor.class}.Mutator implements a mutable vector of fixed width values.  Elements in the
   * vector are accessed by position from the logical start of the vector.  Values should be pushed
@@ -557,11 +539,11 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
      data.setInt((offsetIndex + ${minor.daysOffset}), days);
      data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
    }
-   
+
    protected void set(int index, ${minor.class}Holder holder){
      set(index, holder.months, holder.days, holder.milliseconds);
    }
-   
+
    protected void set(int index, Nullable${minor.class}Holder holder){
      set(index, holder.months, holder.days, holder.milliseconds);
    }
@@ -576,18 +558,18 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
    public void setSafe(int index, Nullable${minor.class}Holder holder){
      setSafe(index, holder.months, holder.days, holder.milliseconds);
    }
-   
+
    public void setSafe(int index, ${minor.class}Holder holder){
      setSafe(index, holder.months, holder.days, holder.milliseconds);
    }
-   
+
    <#elseif (minor.class == "IntervalDay")>
    public void set(int index, int days, int milliseconds){
      int offsetIndex = index * ${type.width};
      data.setInt(offsetIndex, days);
      data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
    }
-   
+
    protected void set(int index, ${minor.class}Holder holder){
      set(index, holder.days, holder.milliseconds);
    }
@@ -601,7 +583,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
      }
      set(index, days, milliseconds);
    }
-   
+
    public void setSafe(int index, ${minor.class}Holder holder){
      setSafe(index, holder.days, holder.milliseconds);
    }
@@ -619,31 +601,31 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
    void set(int index, Nullable${minor.class}Holder holder){
      set(index, holder.start, holder.buffer);
    }
-   
+
    public void setSafe(int index,  Nullable${minor.class}Holder holder){
      setSafe(index, holder.start, holder.buffer);
    }
    public void setSafe(int index,  ${minor.class}Holder holder){
      setSafe(index, holder.start, holder.buffer);
    }
-   
+
    public void setSafe(int index, int start, DrillBuf buffer){
      while(index >= getValueCapacity()) {
        reAlloc();
      }
      set(index, start, buffer);
    }
-   
+
    public void set(int index, int start, DrillBuf buffer){
      data.setBytes(index * ${type.width}, buffer, start, ${type.width});
    }
 
    <#else>
-   
+
    protected void set(int index, ${minor.class}Holder holder){
      set(index, holder.start, holder.buffer);
    }
-   
+
    public void set(int index, Nullable${minor.class}Holder holder){
      set(index, holder.start, holder.buffer);
    }
@@ -651,14 +633,14 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
    public void set(int index, int start, DrillBuf buffer){
      data.setBytes(index * ${type.width}, buffer, start, ${type.width});
    }
-   
+
    public void setSafe(int index, ${minor.class}Holder holder){
      setSafe(index, holder.start, holder.buffer);
    }
    public void setSafe(int index, Nullable${minor.class}Holder holder){
      setSafe(index, holder.start, holder.buffer);
    }
-   
+
    public void setSafe(int index, int start, DrillBuf buffer){
      while(index >= getValueCapacity()) {
        reAlloc();
@@ -675,7 +657,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
    public void generateTestData(int count) {
      setValueCount(count);
      boolean even = true;
-     for(int i =0; i < valueCount; i++, even = !even){
+     for(int i =0; i < getAccessor().getValueCount(); i++, even = !even){
        byte b = even ? Byte.MIN_VALUE : Byte.MAX_VALUE;
        for(int w = 0; w < ${type.width}; w++){
          data.setByte(i + w, b);
@@ -724,7 +706,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
    public void generateTestData(int size) {
      setValueCount(size);
      boolean even = true;
-     for(int i =0; i < valueCount; i++, even = !even){
+     for(int i =0; i < getAccessor().getValueCount(); i++, even = !even){
        if(even){
          set(i, ${minor.boxedType!type.boxedType}.MIN_VALUE);
        }else{
@@ -737,7 +719,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
    public void generateTestDataAlt(int size) {
      setValueCount(size);
      boolean even = true;
-     for(int i =0; i < valueCount; i++, even = !even){
+     for(int i =0; i < getAccessor().getValueCount(); i++, even = !even){
        if(even){
          set(i, (${(minor.javaType!type.javaType)}) 1);
        }else{
@@ -752,7 +734,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   
    public void setValueCount(int valueCount) {
      int currentValueCapacity = getValueCapacity();
-     ${minor.class}Vector.this.valueCount = valueCount;
      int idx = (${type.width} * valueCount);
      while(valueCount > getValueCapacity()) {
        reAlloc();

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
index 0c2341c..9373fc3 100644
--- a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
@@ -50,7 +50,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
 
   private final FieldReader reader = new Nullable${minor.class}ReaderImpl(Nullable${minor.class}Vector.this);
 
-  private int valueCount;
   private final UInt1Vector bits = new UInt1Vector(MaterializedField.create(field + "_bits", Types.required(MinorType.UINT1)), allocator);
   private final ${valuesName} values = new ${minor.class}Vector(field, allocator);
   private final Mutator mutator = new Mutator();
@@ -69,14 +68,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     return Math.min(bits.getValueCapacity(), values.getValueCapacity());
   }
 
-  public int getCurrentValueCount() {
-    return values.getCurrentValueCount();
-  }
-
-  public void setCurrentValueCount(int count) {
-    values.setCurrentValueCount(count);
-  }
-
   @Override
   public DrillBuf[] getBuffers(boolean clear) {
     DrillBuf[] buffers = ObjectArrays.concat(bits.getBuffers(false), values.getBuffers(false), DrillBuf.class);
@@ -91,7 +82,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   
   @Override
   public void clear() {
-    valueCount = 0;
     bits.clear();
     values.clear();
   }
@@ -100,8 +90,9 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     return values.getBufferSize() + bits.getBufferSize();
   }
 
-  public DrillBuf getData(){
-    return values.getData();
+  @Override
+  public DrillBuf getBuffer() {
+    return values.getBuffer();
   }
 
   public ${valuesName} getValuesVector() {
@@ -118,7 +109,7 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   @Override
   public SerializedField getMetadata() {
     return getMetadataBuilder()
-             .setValueCount(valueCount)
+             .setValueCount(getAccessor().getValueCount())
              .setVarByteLength(values.getVarByteLength())
              .setBufferLength(getBufferSize())
              .build();
@@ -152,7 +143,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   @Override
   public int load(int dataBytes, int valueCount, DrillBuf buf){
     clear();
-    this.valueCount = valueCount;
     int loaded = bits.load(valueCount, buf);
     
     // remove bits part of buffer.
@@ -181,13 +171,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   }
 
   <#else>
-  @Override
-  public SerializedField getMetadata() {
-    return getMetadataBuilder()
-             .setValueCount(valueCount)
-             .setBufferLength(getBufferSize())
-             .build();
-  }
 
   @Override
   public void allocateNew() {
@@ -229,7 +212,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   @Override
   public int load(int valueCount, DrillBuf buf){
     clear();
-    this.valueCount = valueCount;
     int loaded = bits.load(valueCount, buf);
     
     // remove bits part of buffer.
@@ -262,7 +244,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   public void transferTo(Nullable${minor.class}Vector target){
     bits.transferTo(target.bits);
     values.transferTo(target.values);
-    target.valueCount = valueCount;
     <#if type.major == "VarLen">
     target.mutator.lastSet = mutator.lastSet;
     </#if>
@@ -272,7 +253,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   public void splitAndTransferTo(int startIndex, int length, Nullable${minor.class}Vector target) {
     bits.splitAndTransferTo(startIndex, length, target.bits);
     values.splitAndTransferTo(startIndex, length, target.values);
-    target.valueCount = length;
     <#if type.major == "VarLen">
     target.mutator.lastSet = length - 1;
     </#if>
@@ -318,7 +298,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   public ${minor.class}Vector convertToRequiredVector(){
     ${minor.class}Vector v = new ${minor.class}Vector(getField().getOtherNullableVersion(), allocator);
     v.data = values.data;
-    v.valueCount = this.valueCount;
     v.data.retain();
     clear();
     return v;
@@ -348,20 +327,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     values.copyFromSafe(fromIndex, thisIndex, from.values);
   }
 
-  public long getDataAddr(){
-    return values.getDataAddr();
-  }
-  
-  public long getBitAddr(){
-    return bits.getDataAddr();
-  }
-  
-  <#if type.major == "VarLen">
-  public long getOffsetAddr(){
-    return values.getOffsetAddr();
-  }
-  </#if>
-  
   public final class Accessor extends BaseDataValueVector.BaseAccessor <#if type.major = "VarLen">implements VariableWidthVector.VariableWidthAccessor</#if> {
 
     final UInt1Vector.Accessor bAccessor = bits.getAccessor();
@@ -391,17 +356,7 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     public long getStartEnd(int index){
       return vAccessor.getStartEnd(index);
     }
-    
-    public long getOffsetAddr(){
-      return values.getOffsetAddr();
-    }
-    </#if>
-    
-
-    
-
 
-    <#if type.major == "VarLen">
     public int getValueLength(int index) {
       return values.getAccessor().getValueLength(index);
     }
@@ -437,7 +392,7 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     </#if>
 
     public int getValueCount(){
-      return valueCount;
+      return bits.getAccessor().getValueCount();
     }
     
     public void reset(){}
@@ -608,14 +563,9 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       <#if type.major == "VarLen">
       fillEmpties(valueCount);
       </#if>
-      Nullable${minor.class}Vector.this.valueCount = valueCount;
       values.getMutator().setValueCount(valueCount);
       bits.getMutator().setValueCount(valueCount);
     }
-    
-    public boolean noNulls(){
-      return valueCount == setCount;
-    }
 
     public void generateTestData(int valueCount){
       bits.getMutator().generateTestDataAlt(valueCount);

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java b/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java
index e20163d..bf84cf9 100644
--- a/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java
@@ -19,6 +19,7 @@
 import java.lang.Override;
 
 import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.RepeatedFixedWidthVector;
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.mortbay.jetty.servlet.Holder;
@@ -76,14 +77,6 @@ public final class Repeated${minor.class}Vector extends BaseValueVector implemen
     return Math.min(values.getValueCapacity(), offsets.getValueCapacity() - 1);
   }
 
-  public int getCurrentValueCount() {
-    return values.getCurrentValueCount();
-  }
-
-  public void setCurrentValueCount(int count) {
-    values.setCurrentValueCount(offsets.getAccessor().get(count));
-  }
-  
   public int getBufferSize(){
     if(accessor.getGroupCount() == 0){
       return 0;
@@ -91,8 +84,8 @@ public final class Repeated${minor.class}Vector extends BaseValueVector implemen
     return offsets.getBufferSize() + values.getBufferSize();
   }
 
-  public DrillBuf getData(){
-      return values.getData();
+  public DrillBuf getBuffer(){
+      return values.getBuffer();
   }
   
   public TransferPair getTransferPair(){
@@ -314,7 +307,7 @@ public final class Repeated${minor.class}Vector extends BaseValueVector implemen
   // variable length vectors, as they should ahve consistent interface as much as possible, if they need to diverge
   // in the future, the interface shold be declared in the respective value vector superclasses for fixed and variable
   // and we should refer to each in the generation template
-  public final class Accessor implements RepeatedFixedWidthVector.RepeatedAccessor{
+  public final class Accessor extends BaseValueVector.BaseAccessor implements RepeatedFixedWidthVector.RepeatedAccessor{
 
     /**
      * Get the elements at the given index.
@@ -405,7 +398,7 @@ public final class Repeated${minor.class}Vector extends BaseValueVector implemen
     }
   }
   
-  public final class Mutator implements RepeatedMutator {
+  public final class Mutator extends BaseValueVector.BaseMutator implements RepeatedMutator {
 
     
     private Mutator(){

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java b/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
index f2b0829..0273304 100644
--- a/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
@@ -82,7 +82,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
   }
 
   public int getBufferSize(){
-    if(valueCount == 0) return 0;
+    if (getAccessor().getValueCount() == 0) return 0;
     return offsetVector.getBufferSize() + data.writerIndex();
   }
   
@@ -99,7 +99,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
   }
 
   public int getCurrentSizeInBytes() {
-    return offsetVector.getAccessor().get(currentValueCount);
+    return offsetVector.getAccessor().get(getAccessor().getValueCount());
   }
   
   /**
@@ -107,21 +107,21 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
    * @return
    */
   public int getVarByteLength(){
+    final int valueCount = getAccessor().getValueCount();
     if(valueCount == 0) return 0;
-    return offsetVector.getAccessor().get(valueCount); 
+    return offsetVector.getAccessor().get(valueCount);
   }
   
   @Override
   public SerializedField getMetadata() {
     return getMetadataBuilder() //
-             .setValueCount(valueCount) //
+             .setValueCount(getAccessor().getValueCount()) //
              .setVarByteLength(getVarByteLength()) //
              .setBufferLength(getBufferSize()) //
              .build();
   }
 
   public int load(int dataBytes, int valueCount, DrillBuf buf){
-    this.valueCount = valueCount;
     if(valueCount == 0){
       allocateNew(0,0);
       return 0;
@@ -159,10 +159,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     return buffers;
   }
   
-  public long getOffsetAddr(){
-    return offsetVector.getDataAddr();
-  }
-  
   public TransferPair getTransferPair(){
     return new TransferImpl(getField());
   }
@@ -179,7 +175,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     this.offsetVector.transferTo(target.offsetVector);
     target.data = data;
     target.data.retain();
-    target.valueCount = valueCount;
     clear();
   }
 
@@ -386,7 +381,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     
     
     public int getValueCount() {
-      return valueCount;
+      return Math.max(offsetVector.getAccessor().getValueCount()-1, 0);
     }
 
     public boolean isNull(int index){
@@ -542,7 +537,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     
     public void setValueCount(int valueCount) {
       int currentByteCapacity = getByteCapacity();
-      ${minor.class}Vector.this.valueCount = valueCount;
       int idx = offsetVector.getAccessor().get(valueCount);
       data.writerIndex(idx);
       if (valueCount > 0 && currentByteCapacity > idx * 2) {

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/expr/GetSetVectorHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/GetSetVectorHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/GetSetVectorHelper.java
index 46bd7f0..a10c207 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/GetSetVectorHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/GetSetVectorHelper.java
@@ -78,11 +78,11 @@ public class GetSetVectorHelper {
         eval.assign(out.getHolder().ref("scale"), vector.invoke("getField").invoke("getScale"));
         eval.assign(out.getHolder().ref("precision"), vector.invoke("getField").invoke("getPrecision"));
         eval.assign(out.getHolder().ref("start"), JExpr.lit(TypeHelper.getSize(type)).mul(indexVariable));
-        eval.assign(out.getHolder().ref("buffer"), vector.invoke("getData"));
+        eval.assign(out.getHolder().ref("buffer"), vector.invoke("getBuffer"));
         return;
       case INTERVAL:{
         JVar start = eval.decl(model.INT, "start", JExpr.lit(TypeHelper.getSize(type)).mul(indexVariable));
-        JVar data = eval.decl(model.ref(DrillBuf.class), "data", vector.invoke("getData"));
+        JVar data = eval.decl(model.ref(DrillBuf.class), "data", vector.invoke("getBuffer"));
         eval.assign(out.getHolder().ref("months"), data.invoke("getInt").arg(start));
         eval.assign(out.getHolder().ref("days"), data.invoke("getInt").arg(start.plus(JExpr.lit(4))));
         eval.assign(out.getHolder().ref("milliseconds"), data.invoke("getInt").arg(start.plus(JExpr.lit(8))));
@@ -90,14 +90,14 @@ public class GetSetVectorHelper {
       }
       case INTERVALDAY: {
         JVar start = eval.decl(model.INT, "start", JExpr.lit(TypeHelper.getSize(type)).mul(indexVariable));
-        eval.assign(out.getHolder().ref("days"), vector.invoke("getData").invoke("getInt").arg(start));
-        eval.assign(out.getHolder().ref("milliseconds"), vector.invoke("getData").invoke("getInt").arg(start.plus(JExpr.lit(4))));
+        eval.assign(out.getHolder().ref("days"), vector.invoke("getBuffer").invoke("getInt").arg(start));
+        eval.assign(out.getHolder().ref("milliseconds"), vector.invoke("getBuffer").invoke("getInt").arg(start.plus(JExpr.lit(4))));
         return;
       }
       case VAR16CHAR:
       case VARBINARY:
       case VARCHAR:
-         eval.assign(out.getHolder().ref("buffer"), vector.invoke("getData"));
+         eval.assign(out.getHolder().ref("buffer"), vector.invoke("getBuffer"));
          JVar se = eval.decl(model.LONG, "startEnd", getValueAccessor.invoke("getStartEnd").arg(indexVariable));
          eval.assign(out.getHolder().ref("start"), JExpr.cast(model._ref(int.class), se));
          eval.assign(out.getHolder().ref("end"), JExpr.cast(model._ref(int.class), se.shr(JExpr.lit(32))));

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
index 759327a..d3f1a30 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReader.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.vector.BaseDataValueVector;
-import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.ValueVector;
 
 import parquet.column.ColumnDescriptor;
@@ -181,7 +180,7 @@ public abstract class ColumnReader<V extends ValueVector> {
     readLengthInBits = 0;
     recordsReadInThisIteration = 0;
     bytesReadInCurrentPass = 0;
-    vectorData = ((BaseDataValueVector) valueVec).getData();
+    vectorData = ((BaseDataValueVector) valueVec).getBuffer();
   }
 
   public int capacity() {

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
index c2af964..a425bc1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
@@ -164,7 +164,7 @@ class FixedByteAlignedReader extends ColumnReader {
     void addNext(int start, int index) {
       int width = Decimal28SparseHolder.WIDTH;
       BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, dataTypeLengthInBytes, schemaElement.getScale());
-      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getData(), index * width, schemaElement.getScale(),
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getBuffer(), index * width, schemaElement.getScale(),
               schemaElement.getPrecision(), Decimal28SparseHolder.nDecimalDigits);
     }
   }
@@ -183,7 +183,7 @@ class FixedByteAlignedReader extends ColumnReader {
     void addNext(int start, int index) {
       int width = Decimal38SparseHolder.WIDTH;
       BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, dataTypeLengthInBytes, schemaElement.getScale());
-      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal38Vector.getData(), index * width, schemaElement.getScale(),
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal38Vector.getBuffer(), index * width, schemaElement.getScale(),
               schemaElement.getPrecision(), Decimal38SparseHolder.nDecimalDigits);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedWidthRepeatedReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedWidthRepeatedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedWidthRepeatedReader.java
index f20d765..05f6417 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedWidthRepeatedReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedWidthRepeatedReader.java
@@ -65,7 +65,7 @@ public class FixedWidthRepeatedReader extends VarLengthColumn {
     bytesReadInCurrentPass = 0;
     valuesReadInCurrentPass = 0;
     pageReader.valuesReadyToRead = 0;
-    dataReader.vectorData = castedRepeatedVector.getMutator().getDataVector().getData();
+    dataReader.vectorData = castedRepeatedVector.getMutator().getDataVector().getBuffer();
     dataReader.valuesReadInCurrentPass = 0;
     repeatedGroupsReadInCurrentPass = 0;
   }
@@ -223,7 +223,7 @@ public class FixedWidthRepeatedReader extends VarLengthColumn {
 
   @Override
   public int capacity() {
-    return castedRepeatedVector.getMutator().getDataVector().getData().capacity();
+    return castedRepeatedVector.getMutator().getDataVector().getBuffer().capacity();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableColumnReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableColumnReader.java
index 16519a8..9e62520 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableColumnReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableColumnReader.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.vector.BaseDataValueVector;
-import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.NullableVectorDefinitionSetter;
 import org.apache.drill.exec.vector.ValueVector;
 
@@ -57,7 +56,7 @@ abstract class NullableColumnReader<V extends ValueVector> extends ColumnReader<
     readLength = 0;
     readLengthInBits = 0;
     recordsReadInThisIteration = 0;
-    vectorData = castedBaseVector.getData();
+    vectorData = castedBaseVector.getBuffer();
 
       // values need to be spaced out where nulls appear in the column
       // leaving blank space for nulls allows for random access to values
@@ -115,9 +114,9 @@ abstract class NullableColumnReader<V extends ValueVector> extends ColumnReader<
         }
         valuesReadInCurrentPass += nullsFound;
 
-        int writerIndex = ((BaseDataValueVector) valueVec).getData().writerIndex();
+        int writerIndex = ((BaseDataValueVector) valueVec).getBuffer().writerIndex();
         if ( dataTypeLengthInBits > 8  || (dataTypeLengthInBits < 8 && totalValuesRead + runLength % 8 == 0)){
-          castedBaseVector.getData().setIndex(0, writerIndex + (int) Math.ceil( nullsFound * dataTypeLengthInBits / 8.0));
+          castedBaseVector.getBuffer().setIndex(0, writerIndex + (int) Math.ceil( nullsFound * dataTypeLengthInBits / 8.0));
         }
         else if (dataTypeLengthInBits < 8){
           rightBitShift += dataTypeLengthInBits * nullsFound;

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
index 8087118..ff1d7f9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
@@ -317,7 +317,7 @@ public class NullableFixedByteAlignedReaders {
     void addNext(int start, int index) {
       int width = NullableDecimal28SparseHolder.WIDTH;
       BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, dataTypeLengthInBytes, schemaElement.getScale());
-      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getData(), index * width, schemaElement.getScale(),
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getBuffer(), index * width, schemaElement.getScale(),
           schemaElement.getPrecision(), NullableDecimal28SparseHolder.nDecimalDigits);
     }
   }
@@ -336,7 +336,7 @@ public class NullableFixedByteAlignedReaders {
     void addNext(int start, int index) {
       int width = NullableDecimal38SparseHolder.WIDTH;
       BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, dataTypeLengthInBytes, schemaElement.getScale());
-      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal38Vector.getData(), index * width, schemaElement.getScale(),
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal38Vector.getBuffer(), index * width, schemaElement.getScale(),
           schemaElement.getPrecision(), NullableDecimal38SparseHolder.nDecimalDigits);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLengthColumnReaders.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLengthColumnReaders.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLengthColumnReaders.java
index 7464f30..4cf5942 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLengthColumnReaders.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLengthColumnReaders.java
@@ -62,14 +62,14 @@ public class VarLengthColumnReaders {
       if (index >= decimal28Vector.getValueCapacity()) {
         return false;
       }
-      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getData(), index * width, schemaElement.getScale(),
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getBuffer(), index * width, schemaElement.getScale(),
               schemaElement.getPrecision(), Decimal28SparseHolder.nDecimalDigits);
       return true;
     }
 
     @Override
     public int capacity() {
-      return decimal28Vector.getData().capacity();
+      return decimal28Vector.getBuffer().capacity();
     }
   }
 
@@ -91,7 +91,7 @@ public class VarLengthColumnReaders {
       if (index >= nullableDecimal28Vector.getValueCapacity()) {
         return false;
       }
-      DecimalUtility.getSparseFromBigDecimal(intermediate, nullableDecimal28Vector.getData(), index * width, schemaElement.getScale(),
+      DecimalUtility.getSparseFromBigDecimal(intermediate, nullableDecimal28Vector.getBuffer(), index * width, schemaElement.getScale(),
               schemaElement.getPrecision(), Decimal28SparseHolder.nDecimalDigits);
       nullableDecimal28Vector.getMutator().setIndexDefined(index);
       return true;
@@ -99,7 +99,7 @@ public class VarLengthColumnReaders {
 
     @Override
     public int capacity() {
-      return nullableDecimal28Vector.getData().capacity();
+      return nullableDecimal28Vector.getBuffer().capacity();
     }
   }
 
@@ -121,14 +121,14 @@ public class VarLengthColumnReaders {
       if (index >= decimal28Vector.getValueCapacity()) {
         return false;
       }
-      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getData(), index * width, schemaElement.getScale(),
+      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getBuffer(), index * width, schemaElement.getScale(),
               schemaElement.getPrecision(), Decimal38SparseHolder.nDecimalDigits);
       return true;
     }
 
     @Override
     public int capacity() {
-      return decimal28Vector.getData().capacity();
+      return decimal28Vector.getBuffer().capacity();
     }
   }
 
@@ -150,7 +150,7 @@ public class VarLengthColumnReaders {
       if (index >= nullableDecimal38Vector.getValueCapacity()) {
         return false;
       }
-      DecimalUtility.getSparseFromBigDecimal(intermediate, nullableDecimal38Vector.getData(), index * width, schemaElement.getScale(),
+      DecimalUtility.getSparseFromBigDecimal(intermediate, nullableDecimal38Vector.getBuffer(), index * width, schemaElement.getScale(),
               schemaElement.getPrecision(), Decimal38SparseHolder.nDecimalDigits);
       nullableDecimal38Vector.getMutator().setIndexDefined(index);
       return true;
@@ -158,7 +158,7 @@ public class VarLengthColumnReaders {
 
     @Override
     public int capacity() {
-      return nullableDecimal38Vector.getData().capacity();
+      return nullableDecimal38Vector.getBuffer().capacity();
     }
   }
 
@@ -203,7 +203,7 @@ public class VarLengthColumnReaders {
 
     @Override
     public int capacity() {
-      return varCharVector.getData().capacity();
+      return varCharVector.getBuffer().capacity();
     }
   }
 
@@ -241,7 +241,7 @@ public class VarLengthColumnReaders {
 
     @Override
     public int capacity() {
-      return vector.getData().capacity();
+      return vector.getBuffer().capacity();
     }
   }
 
@@ -286,7 +286,7 @@ public class VarLengthColumnReaders {
 
     @Override
     public int capacity() {
-      return varBinaryVector.getData().capacity();
+      return varBinaryVector.getBuffer().capacity();
     }
   }
 
@@ -332,7 +332,7 @@ public class VarLengthColumnReaders {
 
     @Override
     public int capacity() {
-      return nullableVarBinaryVector.getData().capacity();
+      return nullableVarBinaryVector.getBuffer().capacity();
     }
 
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
index b45dd5f..0c6097c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -18,44 +18,26 @@
 package org.apache.drill.exec.vector;
 
 import io.netty.buffer.DrillBuf;
-
-import java.util.Iterator;
-
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.MaterializedField;
 
-import com.google.common.collect.Iterators;
 
-public abstract class BaseDataValueVector extends BaseValueVector{
+public abstract class BaseDataValueVector<V extends BaseValueVector<V, A, M>, A extends BaseValueVector.BaseAccessor,
+    M extends BaseValueVector.BaseMutator> extends BaseValueVector<V, A, M> {
 
   protected DrillBuf data;
-  protected int valueCount;
-  protected int currentValueCount;
 
   public BaseDataValueVector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
     this.data = allocator.getEmpty();
   }
 
-  /**
-   * Release the underlying DrillBuf and reset the ValueVector
-   */
   @Override
   public void clear() {
     data.release();
     data = allocator.getEmpty();
   }
 
-  public void setCurrentValueCount(int count) {
-    currentValueCount = count;
-  }
-
-  public int getCurrentValueCount() {
-    return currentValueCount;
-  }
-
-
   @Override
   public DrillBuf[] getBuffers(boolean clear) {
     DrillBuf[] out;
@@ -81,20 +63,8 @@ public abstract class BaseDataValueVector extends BaseValueVector{
     return data.writerIndex();
   }
 
-  @Override
-  public abstract SerializedField getMetadata();
-
-  public DrillBuf getData() {
+  public DrillBuf getBuffer() {
     return data;
   }
 
-  public long getDataAddr() {
-    return data.memoryAddress();
-  }
-
-  @Override
-  public Iterator<ValueVector> iterator() {
-    return Iterators.emptyIterator();
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
index 81d3a86..22f0fe7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
@@ -53,8 +53,15 @@ public abstract class BaseValueVector<V extends BaseValueVector<V, A, M>, A exte
     return getField().clone(ref);
   }
 
+  @Override
+  public SerializedField getMetadata() {
+    return getMetadataBuilder().build();
+  }
+
   protected SerializedField.Builder getMetadataBuilder() {
-    return getField().getAsBuilder();
+    return getField().getAsBuilder()
+        .setValueCount(getAccessor().getValueCount())
+        .setBufferLength(getBufferSize());
   }
 
   public abstract static class BaseAccessor implements ValueVector.Accessor {

http://git-wip-us.apache.org/repos/asf/drill/blob/64e3ec52/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
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 d8bd972..2fc5bf3 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
@@ -42,6 +42,7 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
   private final Accessor accessor = new Accessor();
   private final Mutator mutator = new Mutator();
 
+  private int valueCount;
   private int allocationValueCount = INITIAL_VALUE_ALLOCATION;
   private int allocationMonitor = 0;
 
@@ -55,11 +56,8 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
   }
 
   @Override
-  public SerializedField getMetadata() {
-    return field.getAsBuilder() //
-        .setValueCount(valueCount) //
-        .setBufferLength( (int) Math.ceil(valueCount / 8.0)) //
-        .build();
+  public int getBufferSize() {
+    return getSizeFromCount(valueCount);
   }
 
   private int getSizeFromCount(int valueCount) {


[2/3] drill git commit: DRILL-2719: ValueVector#getBuffers(clear) must consistently clear vectors & retain buffers

Posted by me...@apache.org.
DRILL-2719: ValueVector#getBuffers(clear) must consistently clear vectors & retain buffers


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

Branch: refs/heads/master
Commit: caf779d014f1ddb6ac6b36693760507805b3183a
Parents: a7e9590
Author: Hanifi Gunes <hg...@maprtech.com>
Authored: Wed Apr 8 16:11:34 2015 -0700
Committer: Mehant Baid <me...@gmail.com>
Committed: Thu Apr 16 10:53:34 2015 -0700

----------------------------------------------------------------------
 .../codegen/templates/NullableValueVectors.java    |  5 ++++-
 .../codegen/templates/RepeatedValueVectors.java    |  5 ++++-
 .../codegen/templates/VariableLengthVectors.java   |  6 +++++-
 .../drill/exec/vector/BaseDataValueVector.java     | 17 +++++------------
 .../exec/vector/complex/AbstractMapVector.java     | 10 ++++++++--
 .../exec/vector/complex/RepeatedListVector.java    | 10 +++++++++-
 .../exec/vector/complex/TestEmptyPopulator.java    |  2 ++
 7 files changed, 37 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/caf779d0/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
index b4b837f..0c2341c 100644
--- a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
@@ -79,8 +79,11 @@ public final class ${className} extends BaseDataValueVector implements <#if type
 
   @Override
   public DrillBuf[] getBuffers(boolean clear) {
-    DrillBuf[] buffers = ObjectArrays.concat(bits.getBuffers(clear), values.getBuffers(clear), DrillBuf.class);
+    DrillBuf[] buffers = ObjectArrays.concat(bits.getBuffers(false), values.getBuffers(false), DrillBuf.class);
     if (clear) {
+      for (DrillBuf buffer:buffers) {
+        buffer.retain();
+      }
       clear();
     }
     return buffers;

http://git-wip-us.apache.org/repos/asf/drill/blob/caf779d0/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java b/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java
index c7cf8e6..e20163d 100644
--- a/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java
@@ -285,8 +285,11 @@ public final class Repeated${minor.class}Vector extends BaseValueVector implemen
 
   @Override
   public DrillBuf[] getBuffers(boolean clear) {
-    DrillBuf[] buffers = ObjectArrays.concat(offsets.getBuffers(clear), values.getBuffers(clear), DrillBuf.class);
+    DrillBuf[] buffers = ObjectArrays.concat(offsets.getBuffers(false), values.getBuffers(false), DrillBuf.class);
     if (clear) {
+      for (DrillBuf buffer:buffers) {
+        buffer.retain();
+      }
       clear();
     }
     return buffers;

http://git-wip-us.apache.org/repos/asf/drill/blob/caf779d0/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java b/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
index edb851e..f2b0829 100644
--- a/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java
@@ -148,8 +148,12 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
   
   @Override
   public DrillBuf[] getBuffers(boolean clear) {
-    DrillBuf[] buffers = ObjectArrays.concat(offsetVector.getBuffers(clear), super.getBuffers(clear), DrillBuf.class);
+    DrillBuf[] buffers = ObjectArrays.concat(offsetVector.getBuffers(false), super.getBuffers(false), DrillBuf.class);
     if (clear) {
+      // does not make much sense but we have to retain buffers even when clear is set. refactor this interface.
+      for (DrillBuf buffer:buffers) {
+        buffer.retain();
+      }
       clear();
     }
     return buffers;

http://git-wip-us.apache.org/repos/asf/drill/blob/caf779d0/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
index d48ea99..b45dd5f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -23,7 +23,6 @@ import java.util.Iterator;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
-import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 
 import com.google.common.collect.Iterators;
@@ -36,7 +35,7 @@ public abstract class BaseDataValueVector extends BaseValueVector{
 
   public BaseDataValueVector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
-
+    this.data = allocator.getEmpty();
   }
 
   /**
@@ -44,14 +43,8 @@ public abstract class BaseDataValueVector extends BaseValueVector{
    */
   @Override
   public void clear() {
-    if (data == null) {
-      data = DeadBuf.DEAD_BUFFER;
-    }
-    if (data != DeadBuf.DEAD_BUFFER) {
-      data.release();
-      data = data.getAllocator().getEmpty();
-      valueCount = 0;
-    }
+    data.release();
+    data = allocator.getEmpty();
   }
 
   public void setCurrentValueCount(int count) {
@@ -66,7 +59,7 @@ public abstract class BaseDataValueVector extends BaseValueVector{
   @Override
   public DrillBuf[] getBuffers(boolean clear) {
     DrillBuf[] out;
-    if (valueCount == 0) {
+    if (getBufferSize() == 0) {
       out = new DrillBuf[0];
     } else {
       out = new DrillBuf[]{data};
@@ -82,7 +75,7 @@ public abstract class BaseDataValueVector extends BaseValueVector{
   }
 
   public int getBufferSize() {
-    if (valueCount == 0) {
+    if (getAccessor().getValueCount() == 0) {
       return 0;
     }
     return data.writerIndex();

http://git-wip-us.apache.org/repos/asf/drill/blob/caf779d0/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
index b0783af..78846dc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
@@ -216,9 +216,15 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
   public DrillBuf[] getBuffers(boolean clear) {
     List<DrillBuf> buffers = Lists.newArrayList();
 
-    for (ValueVector v : vectors.values()) {
-      for (DrillBuf buf : v.getBuffers(clear)) {
+    for (ValueVector vector : vectors.values()) {
+      for (DrillBuf buf : vector.getBuffers(false)) {
         buffers.add(buf);
+        if (clear) {
+          buf.retain();
+        }
+      }
+      if (clear) {
+        vector.clear();
       }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/caf779d0/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
index c0f5299..c061029 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
@@ -345,7 +345,15 @@ public class RepeatedListVector extends AbstractContainerVector implements Repea
 
   @Override
   public DrillBuf[] getBuffers(boolean clear) {
-    return ArrayUtils.addAll(offsets.getBuffers(clear), vector.getBuffers(clear));
+    DrillBuf[] buffers = ArrayUtils.addAll(offsets.getBuffers(false), vector.getBuffers(false));
+    if (clear) {
+      // does not make much sense but we have to retain buffers even when clear is set. refactor this interface.
+      for (DrillBuf buffer:buffers) {
+        buffer.retain();
+      }
+      clear();
+    }
+    return buffers;
   }
 
   protected void setVector(ValueVector newVector) {

http://git-wip-us.apache.org/repos/asf/drill/blob/caf779d0/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulator.java
index 8426a6a..f645987 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulator.java
@@ -43,11 +43,13 @@ public class TestEmptyPopulator extends ExecTest {
   private EmptyValuePopulator populator;
 
   private final ByteBuffer buffer = ByteBuffer.allocateDirect(BUF_SIZE);
+  private final ByteBuffer empty = ByteBuffer.allocateDirect(0);
 
 
   @Before
   public void initialize() {
     Mockito.when(allocator.buffer(Mockito.anyInt())).thenReturn(DrillBuf.wrapByteBuffer(buffer));
+    Mockito.when(allocator.getEmpty()).thenReturn(DrillBuf.wrapByteBuffer(empty));
     offsets = new UInt4Vector(null, allocator);
     offsets.allocateNewSafe();
     accessor = offsets.getAccessor();