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/01/28 00:07:28 UTC

[1/3] drill git commit: DRILL-2079: Use AllocationHelper to allocate memory for vectors.

Repository: drill
Updated Branches:
  refs/heads/master 3e3388095 -> b6a387878


DRILL-2079: Use AllocationHelper to allocate memory for vectors.


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

Branch: refs/heads/master
Commit: b6a3878784698cd58015050b29ecc123c841ad5e
Parents: 4976fb9
Author: Mehant Baid <me...@gmail.com>
Authored: Mon Jan 26 19:13:18 2015 -0800
Committer: Mehant Baid <me...@gmail.com>
Committed: Mon Jan 26 22:00:31 2015 -0800

----------------------------------------------------------------------
 .../exec/physical/impl/xsort/PriorityQueueCopierTemplate.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/b6a38787/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
index 17565ed..fe67064 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.AllocationHelper;
 
 public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueCopierTemplate.class);
@@ -123,7 +124,7 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
 
   private void allocateVectors(int targetRecordCount) {
     for (VectorWrapper w: outgoing) {
-      w.getValueVector().allocateNew();
+      AllocationHelper.allocateNew(w.getValueVector(), targetRecordCount);
     }
   }
 


[3/3] drill git commit: DRILL-2072: Fix getGroupCount() in RepeatedMapVector

Posted by me...@apache.org.
DRILL-2072: Fix getGroupCount() in RepeatedMapVector


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

Branch: refs/heads/master
Commit: 47dd6a689178c52035199db572bd93e7d5db9659
Parents: 3e33880
Author: Mehant Baid <me...@gmail.com>
Authored: Mon Jan 26 13:33:35 2015 -0800
Committer: Mehant Baid <me...@gmail.com>
Committed: Mon Jan 26 22:00:31 2015 -0800

----------------------------------------------------------------------
 .../org/apache/drill/exec/vector/complex/RepeatedMapVector.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/47dd6a68/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
----------------------------------------------------------------------
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 c75d0e2..7e1b353 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
@@ -487,7 +487,8 @@ public class RepeatedMapVector extends AbstractContainerVector implements Repeat
 
     @Override
     public int getGroupCount() {
-      return offsets.getAccessor().getValueCount() - 1;
+      final int valueCount = offsets.getAccessor().getValueCount();
+      return valueCount == 0 ? 0 : valueCount - 1;
     }
   }
 


[2/3] drill git commit: DRILL-2078: Remove deprecated VectorAllocator and modify its users

Posted by me...@apache.org.
DRILL-2078: Remove deprecated VectorAllocator and modify its users


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

Branch: refs/heads/master
Commit: 4976fb9e860651895adbe413ca26870932c6ab34
Parents: 47dd6a6
Author: Mehant Baid <me...@gmail.com>
Authored: Mon Jan 26 15:50:26 2015 -0800
Committer: Mehant Baid <me...@gmail.com>
Committed: Mon Jan 26 22:00:31 2015 -0800

----------------------------------------------------------------------
 .../drill/exec/store/hive/HiveRecordReader.java |  6 +--
 .../exec/store/hive/HiveTextRecordReader.java   |  4 +-
 .../impl/aggregate/HashAggTemplate.java         |  2 -
 .../exec/physical/impl/join/MergeJoinBatch.java |  1 -
 .../physical/impl/xsort/ExternalSortBatch.java  |  6 +--
 .../impl/xsort/PriorityQueueCopier.java         |  4 +-
 .../impl/xsort/PriorityQueueCopierTemplate.java | 10 +---
 .../vector/allocator/FixedVectorAllocator.java  | 41 ---------------
 .../RepeatedVariableEstimatedAllocator.java     | 36 -------------
 .../allocator/RepeatedVectorAllocator.java      | 43 ----------------
 .../allocator/VariableEstimatedVector.java      | 42 ----------------
 .../allocator/VariableVectorAllocator.java      | 43 ----------------
 .../exec/vector/allocator/VectorAllocator.java  | 53 --------------------
 13 files changed, 8 insertions(+), 283 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java
index bad7a4e..69c50ea 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java
@@ -45,6 +45,7 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
 import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.util.DecimalUtility;
+import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.DateVector;
@@ -61,7 +62,6 @@ import org.apache.drill.exec.vector.TinyIntVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VarBinaryVector;
 import org.apache.drill.exec.vector.VarCharVector;
-import org.apache.drill.exec.vector.allocator.VectorAllocator;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -281,7 +281,7 @@ public class HiveRecordReader extends AbstractRecordReader {
   @Override
   public int next() {
     for (ValueVector vv : vectors) {
-      VectorAllocator.getAllocator(vv, FIELD_SIZE).alloc(TARGET_RECORD_COUNT);
+      AllocationHelper.allocateNew(vv, TARGET_RECORD_COUNT);
     }
     if (empty) {
       setValueCountAndPopulatePartitionVectors(0);
@@ -427,7 +427,7 @@ public class HiveRecordReader extends AbstractRecordReader {
         size = ((byte[]) selectedPartitionValues.get(i)).length;
       }
 
-      VectorAllocator.getAllocator(vector, size).alloc(recordCount);
+      AllocationHelper.allocateNew(vector, recordCount);
 
       switch(pCat) {
         case BINARY: {

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveTextRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveTextRecordReader.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveTextRecordReader.java
index 2deb7c5..0dd79b2 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveTextRecordReader.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveTextRecordReader.java
@@ -24,11 +24,11 @@ import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.apache.drill.exec.vector.NullableIntVector;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.allocator.VectorAllocator;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
@@ -121,7 +121,7 @@ public class HiveTextRecordReader extends HiveRecordReader {
   @Override
   public int next() {
     for (ValueVector vv : vectors) {
-      VectorAllocator.getAllocator(vv, 50).alloc(TARGET_RECORD_COUNT);
+      AllocationHelper.allocateNew(vv, TARGET_RECORD_COUNT);
     }
     try {
       int recordCount = 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
index 4b8e357..73cadb2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
@@ -57,7 +57,6 @@ import org.apache.drill.exec.vector.ObjectVector;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VariableWidthVector;
-import org.apache.drill.exec.vector.allocator.VectorAllocator;
 
 import com.google.common.collect.Lists;
 
@@ -95,7 +94,6 @@ public abstract class HashAggTemplate implements HashAggregator {
   private IndexPointer outNumRecordsHolder;
   private int numGroupByOutFields = 0; // Note: this should be <= number of group-by fields
 
-  List<VectorAllocator> wsAllocators = Lists.newArrayList();  // allocators for the workspace vectors
   ErrorCollector collector = new ErrorCollectorImpl();
 
   private MaterializedField[] materializedValueFields;

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
index 8a6e1f1..14bc094 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
@@ -53,7 +53,6 @@ import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.allocator.VectorAllocator;
 import org.apache.drill.exec.vector.complex.AbstractContainerVector;
 import org.eigenbase.rel.JoinRelType;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index a22ac98..9026661 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -52,7 +52,6 @@ import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
@@ -62,7 +61,6 @@ import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.util.Utilities;
 import org.apache.drill.exec.vector.CopyUtil;
 import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.allocator.VectorAllocator;
 import org.apache.drill.exec.vector.complex.AbstractContainerVector;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -646,13 +644,11 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
         copier.cleanup();
       }
 
-      List<VectorAllocator> allocators = Lists.newArrayList();
       for (VectorWrapper<?> i : batch) {
         ValueVector v = TypeHelper.getNewVector(i.getField(), copierAllocator);
         outputContainer.add(v);
-        allocators.add(VectorAllocator.getAllocator(v, 110));
       }
-      copier.setup(context, copierAllocator, batch, batchGroupList, outputContainer, allocators);
+      copier.setup(context, copierAllocator, batch, batchGroupList, outputContainer);
     } catch (ClassTransformationException e) {
       throw new RuntimeException(e);
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java
index 4da3c36..d427744 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java
@@ -24,13 +24,11 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.VectorAccessible;
-import org.apache.drill.exec.vector.allocator.VectorAllocator;
 
 public interface PriorityQueueCopier {
   public void setup(FragmentContext context, BufferAllocator allocator, VectorAccessible hyperBatch, List<BatchGroup> batchGroups,
-                    VectorAccessible outgoing, List<VectorAllocator> allocators) throws SchemaChangeException;
+                    VectorAccessible outgoing) throws SchemaChangeException;
   public int next(int targetRecordCount);
-  public List<VectorAllocator> getAllocators();
   public void cleanup();
 
   public static TemplateClassDefinition<PriorityQueueCopier> TEMPLATE_DEFINITION = new TemplateClassDefinition<PriorityQueueCopier>(PriorityQueueCopier.class, PriorityQueueCopierTemplate.class);

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
index dd3d4b5..17565ed 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
@@ -27,7 +27,6 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.vector.allocator.VectorAllocator;
 
 public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueCopierTemplate.class);
@@ -38,20 +37,18 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
   private FragmentContext context;
   private BufferAllocator allocator;
   private VectorAccessible outgoing;
-  private List<VectorAllocator> allocators;
   private int size;
   private int queueSize = 0;
   private int targetRecordCount = ExternalSortBatch.SPILL_TARGET_RECORD_COUNT;
 
   @Override
   public void setup(FragmentContext context, BufferAllocator allocator, VectorAccessible hyperBatch, List<BatchGroup> batchGroups,
-                    VectorAccessible outgoing, List<VectorAllocator> allocators) throws SchemaChangeException {
+                    VectorAccessible outgoing) throws SchemaChangeException {
     this.context = context;
     this.allocator = allocator;
     this.hyperBatch = hyperBatch;
     this.batchGroups = batchGroups;
     this.outgoing = outgoing;
-    this.allocators = allocators;
     this.size = batchGroups.size();
 
     BufferAllocator.PreAllocator preAlloc = allocator.getNewPreAllocator();
@@ -112,11 +109,6 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
     }
   }
 
-  @Override
-  public List<VectorAllocator> getAllocators() {
-    return allocators;
-  }
-
   private void siftUp() {
     int p = queueSize;
     while (p > 0) {

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/FixedVectorAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/FixedVectorAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/FixedVectorAllocator.java
deleted file mode 100644
index e783d02..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/FixedVectorAllocator.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * 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.vector.allocator;
-
-import org.apache.drill.exec.vector.FixedWidthVector;
-
-public class FixedVectorAllocator extends VectorAllocator{
-  FixedWidthVector out;
-
-  public FixedVectorAllocator(FixedWidthVector out) {
-    super();
-    this.out = out;
-  }
-
-  @Override
-  public void alloc(int recordCount){
-    out.allocateNew(recordCount);
-  }
-
-  @Override
-  public String toString() {
-    return "FixedVectorAllocator [out=" + out + ", valueCapacity" + out.getValueCapacity() + "]";
-  }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/RepeatedVariableEstimatedAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/RepeatedVariableEstimatedAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/RepeatedVariableEstimatedAllocator.java
deleted file mode 100644
index c234687..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/RepeatedVariableEstimatedAllocator.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * 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.vector.allocator;
-
-import org.apache.drill.exec.vector.RepeatedVariableWidthVector;
-
-public class RepeatedVariableEstimatedAllocator extends VectorAllocator{
-  RepeatedVariableWidthVector out;
-  int avgWidth;
-
-  public RepeatedVariableEstimatedAllocator(RepeatedVariableWidthVector out, int avgWidth) {
-    super();
-    this.out = out;
-    this.avgWidth = avgWidth;
-  }
-
-  @Override
-  public void alloc(int recordCount){
-    out.allocateNew(avgWidth * recordCount, recordCount, recordCount);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/RepeatedVectorAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/RepeatedVectorAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/RepeatedVectorAllocator.java
deleted file mode 100644
index b4428df..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/RepeatedVectorAllocator.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.vector.allocator;
-
-import org.apache.drill.exec.vector.RepeatedVariableWidthVector;
-
-class RepeatedVectorAllocator extends VectorAllocator{
-  RepeatedVariableWidthVector in;
-  RepeatedVariableWidthVector out;
-
-  public RepeatedVectorAllocator(RepeatedVariableWidthVector in, RepeatedVariableWidthVector out) {
-    super();
-    this.in = in;
-    this.out = out;
-  }
-
-  @Override
-  public void alloc(int recordCount){
-    out.allocateNew(in.getByteCapacity(), in.getAccessor().getValueCount(), in.getAccessor().getValueCount());
-  }
-
-  @Override
-  public String toString() {
-    return "RepeatedVectorAllocator [out=" + out + ", valueCapacity" + out.getValueCapacity() + ", bytesCapacity" + out.getByteCapacity() + "]";
-  }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VariableEstimatedVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VariableEstimatedVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VariableEstimatedVector.java
deleted file mode 100644
index 815a531..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VariableEstimatedVector.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * 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.vector.allocator;
-
-import org.apache.drill.exec.vector.VariableWidthVector;
-
-public class VariableEstimatedVector extends VectorAllocator{
-  VariableWidthVector out;
-  int avgWidth;
-
-  public VariableEstimatedVector(VariableWidthVector out, int avgWidth) {
-    super();
-    this.out = out;
-    this.avgWidth = avgWidth;
-  }
-
-  @Override
-  public void alloc(int recordCount){
-    out.allocateNew(avgWidth * recordCount, recordCount);
-  }
-
-  @Override
-  public String toString() {
-    return "VariableEstimatedVector [out=" + out + ", valueCapacity" + out.getValueCapacity() + ", bytesCapacity" + out.getByteCapacity() + "]";
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VariableVectorAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VariableVectorAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VariableVectorAllocator.java
deleted file mode 100644
index 7934ecb..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VariableVectorAllocator.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.vector.allocator;
-
-import org.apache.drill.exec.vector.VariableWidthVector;
-
-class VariableVectorAllocator extends VectorAllocator{
-  VariableWidthVector in;
-  VariableWidthVector out;
-
-  public VariableVectorAllocator(VariableWidthVector in, VariableWidthVector out) {
-    super();
-    this.in = in;
-    this.out = out;
-  }
-
-  @Override
-  public void alloc(int recordCount){
-    out.allocateNew(in.getByteCapacity(), recordCount);
-  }
-
-  @Override
-  public String toString() {
-    return "VariableVectorAllocator [out=" + out + ", valueCapacity" + out.getValueCapacity() + ", bytesCapacity" + out.getByteCapacity() + "]";
-  }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4976fb9e/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VectorAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VectorAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VectorAllocator.java
deleted file mode 100644
index eb01bef..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/allocator/VectorAllocator.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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.vector.allocator;
-
-import org.apache.drill.exec.vector.FixedWidthVector;
-import org.apache.drill.exec.vector.RepeatedVariableWidthVector;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.VariableWidthVector;
-
-@Deprecated
-public abstract class VectorAllocator{
-  public abstract void alloc(int recordCount);
-
-//  public static VectorAllocator getAllocator(ValueVector in, ValueVector outgoing){
-//    if(outgoing instanceof FixedWidthVector){
-//      return new FixedVectorAllocator((FixedWidthVector) outgoing);
-//    }else if(outgoing instanceof VariableWidthVector && in instanceof VariableWidthVector){
-//      return new VariableVectorAllocator( (VariableWidthVector) in, (VariableWidthVector) outgoing);
-//    } else if (outgoing instanceof RepeatedVariableWidthVector && in instanceof RepeatedVariableWidthVector) {
-//      return new RepeatedVectorAllocator((RepeatedVariableWidthVector) in, (RepeatedVariableWidthVector) outgoing);
-//    }else{
-//      throw new UnsupportedOperationException();
-//    }
-//  }
-
-  @Deprecated
-  public static VectorAllocator getAllocator(ValueVector outgoing, int averageBytesPerVariable){
-    if(outgoing instanceof FixedWidthVector){
-      return new FixedVectorAllocator((FixedWidthVector) outgoing);
-    }else if(outgoing instanceof VariableWidthVector){
-      return new VariableEstimatedVector( (VariableWidthVector) outgoing, averageBytesPerVariable);
-    }else if (outgoing instanceof RepeatedVariableWidthVector) {
-      return new RepeatedVariableEstimatedAllocator((RepeatedVariableWidthVector) outgoing, averageBytesPerVariable);
-    } else {
-      throw new UnsupportedOperationException();
-    }
-  }
-}
\ No newline at end of file