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 <meha...@gmail.com> Authored: Mon Jan 26 15:50:26 2015 -0800 Committer: Mehant Baid <meha...@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