From 51558954489e9015f604a0d017c519664eb7d55a Mon Sep 17 00:00:00 2001 From: Chris Westin Date: Fri, 13 Feb 2015 15:30:24 -0800 Subject: [PATCH 1/4] DRILL-1942: Improve off-heap memory usage tracking This is a reimplementation of the allocators to make tracking better. The re-implementation includes support for JMX MBeans for on-the-fly memory statistics inspection. The new classes are BaseAllocator, RootAllocator, and ChildAllocator. The BufferAllocator interface has been modified a bit. In addition to that, the new allocator flattens the use of buffers in slices, so that a DrillBuf is always referring to an UnsafeDirectLittleEndian, and never to another DrillBuf; this simplifies figuring out what's going on when something goes wrong. The new allocator (BaseAllocator, and its derived classes RootAllocator and ChildAllocator) treats sharing buffers in the same way as slicing buffers, and sharing and ownership transfer of buffers and their slices both work. Record batch and value vector classes previously had a mandatory cleanup() method. These classes have been made to include AutoCloseable as a base class, and cleanup() has been renamed to close(). This was done to get additional help from tools to find places that we need to close() things. This allocator detects a lot more problems than the old one did; fixed many leaks in various locations. AbstractRecordBatch: - cleanup() -> close() - hygiene AbstractSingleRecordBatch: - cleanup() -> close() - hygiene Accounter: - hygiene - TODOs AllocatorLedger: - created private interface for child allocators to communicate with parents AllocationReservation: - created AtomicRemainder: - TODOs BaseAllocator: - created common base class for buffer allocators BaseDataValueVector - released the data buffer in close() BitVector - fixed a bug in splitAndTransferTo(), which was missing a retain() call after slicing a buffer for the transfer target. BootStrapContext: - hygiene BroadcastSenderRootExec: - cleanup() => close() - hygiene BufferAllocator: - replace preallocator with createReservation() - Closeable -> AutoCloseable, and add throws clause - required minor changes to several other files - created DrillCloseables and DrillAutoCloseables to make this easy - renamed the second form of BufferAllocator.takeOwnership to shareOwnership - clean up interface definition - no abstract needed - hygiene - some javadoc - TODOs ChildAllocator - created DataServer - handle failed data transfers due to cancelled closed fragments DrillAutoCloseables: - add comment that null argument is supported - fix a typo in a comment Drillbit: - sychhronize ShutdownThread.run()'s action against a static to avoid concurrent shutdown problems with metrics by serializing the shutdown of multiple drillbits running in the same JVM. DrillBuf: - add some finals - minor formatting - fix spelling mixtake - TODOs DrillConfig: - hygiene DrillRuleSets: - can't reuse query context from past queries ExecConstants - debugging property for memory to ExecConstants - added ALLOCATION_POLICY property name ExternalSortBatch: - only get allocator once, outside loop body - release buffers obtained from getBuffers() FakeAllocator: - switch from preallocator to AllocationReservation - private logger - TODOs FilterRecordBatch: - made some variables private FragmentContext: - getAllocator() shouldn't be @Deprecated - made logger private - TODOs ForemanException: - removed unused fromThrowable() - commented out unused logger - formatting FragmentSetupException: - leave out the finals HashAggTemplate: - cleanup() -> close() - hygiene MappifyUtility: - remove unused imports MergeJoinBatchBuilder: - replace preallocator with AllocationReservation - implement close() to clean up reservation MSortTemplate: - made some variables private NestedConfig: - hygiene OperatorContext: - removed unused applyFragmentLimits member - made logger private - TODOs OutOfMemoryException: - add serialVersionUID to silence warning - minor formatting OutOfMemoryRuntimeException: - add serialVersionUID to silence warning - minor formatting PartitionSenderRootExec: - cleanup() -> close() - hygiene PooledByteBufAllocatorL: - finals - formatting - make logger private (and then comment out because it is unused) PrintingResultsListener: - TODOs PriorityQueueCopierTemplate: - removed unnecessary use of the pre-allocator - made some members private PriorityQueueTemplate: - removed unnecessary use of preallocator - made some members private RecordBatch: - interface now extends AutoCloseable instead of declaring it's own cleanup() - hygiene - TODOs RootAllocator: - created replacement for TopLevelAllocator ScreenCreator: - clean() => close() - hygiene SimpleRootExec: - hygiene - fixup RecordBatch close() SingleSenderCreator: - cleanup() => close() - hygiene SortRecordBatchBuilder: - replace preallocation with AllocationReservation - implements AutoCloseable - implement close() to close reservation - clear batches list after closing items on it - formatting - hygiene StackTrace: - DRILL-2624: fixed numeric formatting for line numbers TestBaseAllocator: - created new test for allocators, unit tests them in isolation TopLevelAllocator: - replace preallocator with AllocationReservation implementation - eliminated @Deprecated constructors - changed many other files to use remaining constructor - eliminate private constructor - add finals, private - formatting - TODOs UnionAllRecordBatch: - added missing null check to clearCurrentRecordBatch() VectorAccessible: - hygiene (mostly comments) ExternalSortBatch: ExpressionInterpreterTest: FilterRecordBatch: HashAggBatch: HashJoinBatch: IteratorValidatorBatchIterator: LimitRecordBatch: MergeJoinBatch: MergingRecordBatch: OrderedPartitionRecordBatch: ProducerConsumerBatch: RemovingRecordBatch: ScanBatch: SortBatch: StreamingAggBatch: StreamingWindowFrameRecordBatch: TopNBatch: TraceRecordBatch: UnionAllRecordBatch: UnorderedReceiverBatch: WriterRecordBatch: - cleanup() -> close() - close builders that use AllocationReservation - hygiene TestAllocators TestSimpleFragmentRun TestSimpleFunctions TestStringFunctions TestLoad TestRepeated TestStringFunctions ... many others... - modify to use new allocator - some hygiene ValueVector and RecordWriter templates: - added clauses to release buffers before references are overwritten - eliminated common subexpressions and repeated getAccessor()/getMutator() calls in loops - hygiene --- .../drill/common/AutoCloseablePointer.java | 112 ++ .../drill/common/DeferredException.java | 42 +- .../drill/common/DrillAutoCloseables.java | 50 + .../apache/drill/common/DrillCloseables.java | 55 + .../apache/drill/common/HistoricalLog.java | 179 ++ .../org/apache/drill/common/StackTrace.java | 40 +- .../drill/common/config/DrillConfig.java | 58 +- .../drill/common/config/NestedConfig.java | 43 +- .../exec/store/hbase/HBaseRecordReader.java | 40 +- .../exec/store/hive/HiveRecordReader.java | 9 +- .../exec/store/mongo/MongoRecordReader.java | 4 +- .../templates/AbstractFieldWriter.java | 80 +- .../main/codegen/templates/BaseWriter.java | 33 +- .../codegen/templates/ComplexWriters.java | 73 +- .../codegen/templates/FixedValueVectors.java | 291 +-- .../templates/JsonOutputRecordWriter.java | 3 +- .../main/codegen/templates/ListWriters.java | 136 +- .../main/codegen/templates/MapWriters.java | 119 +- .../templates/NullableValueVectors.java | 184 +- .../templates/ParquetOutputRecordWriter.java | 3 +- .../templates/RepeatedValueVectors.java | 175 +- .../templates/StringOutputRecordWriter.java | 3 +- .../templates/VariableLengthVectors.java | 258 +-- .../main/java/io/netty/buffer/DrillBuf.java | 607 ++++-- .../java/io/netty/buffer/FakeAllocator.java | 164 -- .../netty/buffer/PooledByteBufAllocatorL.java | 54 +- .../buffer/UnsafeDirectLittleEndian.java | 318 ++-- .../org/apache/drill/exec/ExecConstants.java | 2 + .../drill/exec/TestMemoryRetention.java | 20 +- .../cache/AbstractStreamSerializable.java | 6 +- .../LoopedAbstractDrillSerializable.java | 10 +- .../cache/VectorAccessibleSerializable.java | 31 +- .../apache/drill/exec/client/DrillClient.java | 68 +- .../org/apache/drill/exec/client/DumpCat.java | 24 +- .../exec/client/PrintingResultsListener.java | 29 +- .../exception/FragmentSetupException.java | 9 +- .../exec/expr/fn/impl/MappifyUtility.java | 3 - .../exec/expr/fn/impl/StringFunctions.java | 44 +- .../apache/drill/exec/memory/Accountor.java | 462 ----- .../drill/exec/memory/AllocationPolicy.java | 38 + .../exec/memory/AllocationPolicyAgent.java | 69 + .../exec/memory/AllocationReservation.java | 152 ++ .../exec/memory/AllocatorClosedException.java | 31 + .../drill/exec/memory/AllocatorOwner.java | 36 + .../exec/memory/AllocatorsStatsMXBean.java | 34 + .../drill/exec/memory/AtomicRemainder.java | 216 --- .../drill/exec/memory/BaseAllocator.java | 1673 +++++++++++++++++ .../drill/exec/memory/BufferAllocator.java | 151 +- .../drill/exec/memory/BufferLedger.java | 94 + .../exec/memory/ChainedAllocatorOwner.java | 54 + .../drill/exec/memory/ChildAllocator.java | 47 + .../exec/memory/OutOfMemoryException.java | 14 +- .../memory/OutOfMemoryRuntimeException.java | 9 +- .../drill/exec/memory/RootAllocator.java | 119 ++ .../exec/memory/RootAllocatorStatsMXBean.java | 47 + .../drill/exec/memory/TopLevelAllocator.java | 401 ---- .../apache/drill/exec/ops/BufferManager.java | 9 +- .../drill/exec/ops/FragmentContext.java | 54 +- .../drill/exec/ops/OperatorContext.java | 4 +- .../drill/exec/ops/OperatorContextImpl.java | 55 +- .../apache/drill/exec/ops/QueryContext.java | 7 +- .../exec/physical/impl/BaseRootExec.java | 43 +- .../drill/exec/physical/impl/ImplCreator.java | 5 +- .../drill/exec/physical/impl/ScanBatch.java | 83 +- .../exec/physical/impl/ScreenCreator.java | 20 +- .../physical/impl/SingleSenderCreator.java | 24 +- .../impl/TopN/PriorityQueueTemplate.java | 13 +- .../exec/physical/impl/TopN/TopNBatch.java | 39 +- .../exec/physical/impl/WriterRecordBatch.java | 35 +- .../physical/impl/aggregate/HashAggBatch.java | 21 +- .../impl/aggregate/HashAggTemplate.java | 52 +- .../impl/aggregate/StreamingAggBatch.java | 29 +- .../BroadcastSenderRootExec.java | 7 +- .../impl/filter/FilterRecordBatch.java | 17 +- .../physical/impl/join/HashJoinBatch.java | 17 +- .../physical/impl/join/MergeJoinBatch.java | 24 +- .../impl/join/MergeJoinBatchBuilder.java | 59 +- .../impl/join/NestedLoopJoinBatch.java | 11 +- .../physical/impl/limit/LimitRecordBatch.java | 38 +- .../mergereceiver/MergingRecordBatch.java | 57 +- .../OrderedPartitionRecordBatch.java | 148 +- .../PartitionSenderRootExec.java | 14 +- .../impl/producer/ProducerConsumerBatch.java | 7 +- .../exec/physical/impl/sort/SortBatch.java | 5 +- .../impl/sort/SortRecordBatchBuilder.java | 107 +- .../impl/svremover/RemovingRecordBatch.java | 64 +- .../physical/impl/trace/TraceRecordBatch.java | 25 +- .../impl/union/UnionAllRecordBatch.java | 67 +- .../UnorderedReceiverBatch.java | 14 +- .../IteratorValidatorBatchIterator.java | 18 +- .../impl/window/WindowFrameRecordBatch.java | 8 +- .../impl/xsort/ExternalSortBatch.java | 262 +-- .../physical/impl/xsort/MSortTemplate.java | 12 +- .../impl/xsort/PriorityQueueCopier.java | 16 +- .../xsort/PriorityQueueCopierTemplate.java | 23 +- .../exec/planner/logical/DrillRuleSets.java | 127 +- .../exec/planner/logical/DrillWindowRel.java | 1 - .../exec/record/AbstractRecordBatch.java | 12 +- .../record/AbstractSingleRecordBatch.java | 4 +- .../drill/exec/record/RawFragmentBatch.java | 14 +- .../apache/drill/exec/record/RecordBatch.java | 47 +- .../drill/exec/record/RecordBatchLoader.java | 15 +- .../drill/exec/record/VectorAccessible.java | 26 + .../drill/exec/record/WritableBatch.java | 104 +- .../record/selection/SelectionVector2.java | 29 +- .../record/selection/SelectionVector4.java | 10 +- .../org/apache/drill/exec/rpc/RpcDecoder.java | 4 +- .../drill/exec/rpc/data/DataServer.java | 24 +- .../drill/exec/rpc/user/QueryDataBatch.java | 9 +- .../exec/rpc/user/QueryResultHandler.java | 6 +- .../drill/exec/rpc/user/UserClient.java | 7 +- .../drill/exec/server/BootStrapContext.java | 26 +- .../apache/drill/exec/server/Drillbit.java | 16 +- .../exec/store/AbstractRecordReader.java | 2 +- .../apache/drill/exec/store/RecordReader.java | 15 +- .../exec/store/avro/AvroRecordReader.java | 2 +- .../store/easy/json/JSONRecordReader.java | 33 +- .../exec/store/easy/json/JsonProcessor.java | 2 - .../easy/json/reader/BaseJsonProcessor.java | 11 +- .../compliant/CompliantTextRecordReader.java | 2 +- .../exec/store/mock/MockRecordReader.java | 51 +- .../exec/store/mock/MockScanBatchCreator.java | 10 +- .../columnreaders/ParquetRecordReader.java | 44 +- .../store/parquet2/DrillParquetReader.java | 36 +- .../exec/store/pojo/PojoRecordReader.java | 6 +- .../drill/exec/store/sys/MemoryIterator.java | 6 +- .../store/text/DrillTextRecordReader.java | 40 +- .../testing/ExecutionControlsInjector.java | 9 +- .../drill/exec/vector/AllocationHelper.java | 13 +- .../exec/vector/BaseDataValueVector.java | 21 +- .../drill/exec/vector/BaseValueVector.java | 16 +- .../apache/drill/exec/vector/BitVector.java | 33 +- .../apache/drill/exec/vector/ValueVector.java | 13 +- .../complex/AbstractContainerVector.java | 5 +- .../vector/complex/AbstractMapVector.java | 55 +- .../drill/exec/vector/complex/MapVector.java | 26 +- .../vector/complex/RepeatedListVector.java | 4 +- .../vector/complex/RepeatedMapVector.java | 95 +- .../complex/fn/DrillBufInputStream.java | 5 +- .../exec/vector/complex/fn/JsonReader.java | 7 +- .../complex/impl/AbstractBaseWriter.java | 16 +- .../complex/impl/ComplexWriterImpl.java | 29 +- .../complex/impl/VectorContainerWriter.java | 27 +- .../vector/complex/writer/FieldWriter.java | 6 +- .../work/batch/AbstractDataCollector.java | 16 +- .../exec/work/batch/BaseRawBatchBuffer.java | 9 + .../work/batch/SpoolingRawBatchBuffer.java | 34 +- .../drill/exec/work/foreman/Foreman.java | 13 +- .../exec/work/foreman/ForemanException.java | 24 +- .../exec/work/fragment/FragmentExecutor.java | 6 +- .../exec/work/fragment/FragmentManager.java | 30 +- .../work/fragment/NonRootFragmentManager.java | 18 +- .../work/fragment/RootFragmentManager.java | 7 +- .../java/org/apache/drill/BaseTestQuery.java | 27 +- .../java/org/apache/drill/PlanningBase.java | 15 +- .../java/org/apache/drill/QueryTestUtil.java | 3 +- .../apache/drill/TestAllocationException.java | 20 +- .../java/org/apache/drill/TestTpchLimit0.java | 6 + .../org/apache/drill/TestTpchPlanning.java | 52 +- .../apache/drill/exec/client/DumpCatTest.java | 12 +- .../drill/exec/expr/ExpressionTest.java | 7 +- .../fn/impl/TestByteComparisonFunctions.java | 13 +- .../drill/exec/fn/impl/TestCastFunctions.java | 5 - .../drill/exec/fn/impl/TestMathFunctions.java | 11 +- .../exec/fn/impl/TestNewMathFunctions.java | 12 +- .../exec/fn/impl/TestRepeatedFunction.java | 15 +- .../drill/exec/memory/TestAllocators.java | 96 +- .../drill/exec/memory/TestBaseAllocator.java | 645 +++++++ .../drill/exec/memory/TestEndianess.java | 25 +- .../exec/physical/impl/SimpleRootExec.java | 2 +- .../exec/physical/impl/TestCastFunctions.java | 44 +- .../impl/TestComparisonFunctions.java | 5 +- .../impl/TestImplicitCastFunctions.java | 9 +- .../exec/physical/impl/TestOptiqPlans.java | 20 +- .../physical/impl/TestSimpleFragmentRun.java | 38 +- .../physical/impl/TestSimpleFunctions.java | 21 +- .../physical/impl/TestStringFunctions.java | 8 +- .../exec/physical/impl/TestUnionExchange.java | 15 +- .../drill/exec/physical/impl/agg/TestAgg.java | 10 +- .../physical/impl/common/TestHashTable.java | 10 +- .../impl/filter/TestSimpleFilter.java | 18 +- .../exec/physical/impl/join/TestHashJoin.java | 26 +- .../physical/impl/join/TestMergeJoin.java | 18 +- .../physical/impl/limit/TestSimpleLimit.java | 12 +- .../mergereceiver/TestMergingReceiver.java | 84 +- .../impl/project/TestSimpleProjection.java | 9 +- .../physical/impl/sort/TestSimpleSort.java | 7 +- .../impl/trace/TestTraceMultiRecordBatch.java | 12 +- .../impl/trace/TestTraceOutputDump.java | 31 +- .../physical/impl/union/TestSimpleUnion.java | 18 +- .../exec/physical/impl/writer/TestWriter.java | 19 +- .../drill/exec/record/vector/TestLoad.java | 42 +- .../exec/record/vector/TestValueVector.java | 361 +++- .../drill/exec/server/DrillClientFactory.java | 15 +- .../exec/server/TestDrillbitResilience.java | 9 +- .../exec/store/TestDirectCodecFactory.java | 10 +- .../exec/store/json/TestJsonRecordReader.java | 16 +- .../drill/exec/store/parquet/FieldInfo.java | 1 - .../parquet/ParquetRecordReaderTest.java | 17 +- .../drill/exec/testing/TestResourceLeak.java | 29 +- .../exec/vector/TestSplitAndTransfer.java | 55 +- .../vector/complex/TestEmptyPopulation.java | 7 +- .../complex/writer/TestComplexTypeReader.java | 59 +- .../complex/writer/TestComplexTypeWriter.java | 4 +- .../vector/complex/writer/TestJsonReader.java | 6 +- .../vector/complex/writer/TestRepeated.java | 25 +- .../drill/jdbc/impl/DrillConnectionImpl.java | 42 +- 207 files changed, 7456 insertions(+), 4459 deletions(-) create mode 100644 common/src/main/java/org/apache/drill/common/AutoCloseablePointer.java create mode 100644 common/src/main/java/org/apache/drill/common/DrillAutoCloseables.java create mode 100644 common/src/main/java/org/apache/drill/common/DrillCloseables.java create mode 100644 common/src/main/java/org/apache/drill/common/HistoricalLog.java delete mode 100644 exec/java-exec/src/main/java/io/netty/buffer/FakeAllocator.java delete mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java delete mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferLedger.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocator.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorStatsMXBean.java delete mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java diff --git a/common/src/main/java/org/apache/drill/common/AutoCloseablePointer.java b/common/src/main/java/org/apache/drill/common/AutoCloseablePointer.java new file mode 100644 index 00000000000..3a0ac4a2db5 --- /dev/null +++ b/common/src/main/java/org/apache/drill/common/AutoCloseablePointer.java @@ -0,0 +1,112 @@ +/** + * 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.common; + +/** + * A class similar to Pointer<>, but with features unique to holding + * AutoCloseable pointers. The AutoCloseablePointer<> must be closed + * when it will no longer be used. + * + *

If you're familiar with C++/Boost's shared_ptr<>, you might recognize + * some of the features here.

+ * + * @param type of the pointer + */ +public final class AutoCloseablePointer implements AutoCloseable { + private T pointer; + + /** + * Constructor for a null-valued pointer. + */ + public AutoCloseablePointer() { + pointer = null; + } + + /** + * Constructor for a pointer value. + * + * @param pointer the initial pointer value + */ + public AutoCloseablePointer(final T pointer) { + this.pointer = pointer; + } + + @Override + public void close() throws Exception { + assign(null); + } + + /** + * Get the raw pointer out for use. + * + * @return the raw pointer + */ + public T get() { + return pointer; + } + + /** + * The caller adopts the pointer; the holder is set to + * null, and will no longer be responsible for close()ing this pointer. + * + * @return the pointer being adopted; may be null + */ + public T adopt() { + final T p = pointer; + pointer = null; + return p; + } + + /** + * Assign a new pointer to this holder. Any currently held pointer + * will first be closed. If closing the currently held pointer throws + * an exception, the new pointer is still assigned, and the holder must still + * be closed to close that. + * + *

This makes it convenient to assign a new pointer without having to check + * for a previous value and worry about cleaning it up; this does all that.

+ * + * @param newP the new pointer to hold + * @throws Exception any exception thrown by closing the currently held + * pointer + */ + public void assign(final T newP) throws Exception { + try { + if (pointer != null) { + pointer.close(); + } + } finally { + pointer = newP; + } + } + + /** + * Like {@link #assign(AutoCloseable)}, except that any exception thrown + * by closing the previously held pointer is wrapped with (an unchecked) + * {@link RuntimeException}. + * + * @param newP the new pointer to hold + */ + public void assignNoChecked(final T newP) { + try { + assign(newP); + } catch(final Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/common/src/main/java/org/apache/drill/common/DeferredException.java b/common/src/main/java/org/apache/drill/common/DeferredException.java index c7111a9c042..aa264189f76 100644 --- a/common/src/main/java/org/apache/drill/common/DeferredException.java +++ b/common/src/main/java/org/apache/drill/common/DeferredException.java @@ -18,6 +18,7 @@ package org.apache.drill.common; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; /** * Collects one or more exceptions that may occur, using @@ -30,8 +31,35 @@ *

This class is thread safe. */ public class DeferredException implements AutoCloseable { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DeferredException.class); + private Exception exception = null; private boolean isClosed = false; + private final Supplier exceptionSupplier; + + /** + * Constructor. + */ + public DeferredException() { + this(null); + } + + /** + * Constructor. This constructor accepts a Supplier that can be used + * to create the root exception iff any other exceptions are added. For + * example, in a series of resources closures in a close(), if any of + * the individual closures fails, the root exception should come from + * the current class, not from the first subordinate close() to fail. + * This can be used to provide an exception in that case which will be + * the root exception; the subordinate failed close() will be added to + * that exception as a suppressed exception. + * + * @param exceptionSupplier lazily supplies what will be the root exception + * if any exceptions are added + */ + public DeferredException(Supplier exceptionSupplier) { + this.exceptionSupplier = exceptionSupplier; + } /** * Add an exception. If this is the first exception added, it will be the one @@ -47,6 +75,15 @@ public void addException(final Exception exception) { Preconditions.checkState(!isClosed); if (this.exception == null) { + if (exceptionSupplier == null) { + this.exception = exception; + } else { + this.exception = exceptionSupplier.get(); + if (this.exception == null) { + this.exception = new RuntimeException("Missing root exception"); + } + this.exception.addSuppressed(exception); + } this.exception = exception; } else { this.exception.addSuppressed(exception); @@ -93,7 +130,7 @@ public synchronized Exception getAndClear() { * * @throws Exception */ - public synchronized void throwAndClear() throws Exception{ + public synchronized void throwAndClear() throws Exception { final Exception e = getAndClear(); if (e != null) { throw e; @@ -123,6 +160,9 @@ public void suppressingClose(final AutoCloseable autoCloseable) { autoCloseable.close(); } catch(final Exception e) { addException(e); + } catch(final Error e) { + logger.warn("caught Error {}", e); + throw e; } } } diff --git a/common/src/main/java/org/apache/drill/common/DrillAutoCloseables.java b/common/src/main/java/org/apache/drill/common/DrillAutoCloseables.java new file mode 100644 index 00000000000..11fb9a8f579 --- /dev/null +++ b/common/src/main/java/org/apache/drill/common/DrillAutoCloseables.java @@ -0,0 +1,50 @@ +/** + * 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.common; + +/** + * Provides functionality comparable to Guava's Closeables for AutoCloseables. + */ +public class DrillAutoCloseables { + /** + * Constructor. Prevents construction for class of static utilities. + */ + private DrillAutoCloseables() { + } + + /** + * close() an {@see java.lang.AutoCloseable} without throwing a (checked) + * {@see java.lang.Exception}. This wraps the close() call with a + * try-catch that will rethrow an Exception wrapped with a + * {@see java.lang.RuntimeException}, providing a way to call close() + * without having to do the try-catch everywhere or propagate the Exception. + * + * @param closeable the AutoCloseable to close; may be null + * @throws RuntimeException if an Exception occurs; the Exception is + * wrapped by the RuntimeException + */ + public static void closeNoChecked(final AutoCloseable autoCloseable) { + if (autoCloseable != null) { + try { + autoCloseable.close(); + } catch(final Exception e) { + throw new RuntimeException("Exception while closing", e); + } + } + } +} diff --git a/common/src/main/java/org/apache/drill/common/DrillCloseables.java b/common/src/main/java/org/apache/drill/common/DrillCloseables.java new file mode 100644 index 00000000000..289066b32d9 --- /dev/null +++ b/common/src/main/java/org/apache/drill/common/DrillCloseables.java @@ -0,0 +1,55 @@ +/** + * 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.common; + +import java.io.Closeable; +import java.io.IOException; + +/** + * Provides additional functionality to Guava's Closeables. + */ +public class DrillCloseables { + /** + * Constructor. Prevents construction for class of static utilities. + */ + private DrillCloseables() { + } + + /** + * Close() a {@see java.io.Closeable} without throwing a (checked) + * {@see java.io.IOException}. This wraps the close() call with a + * try-catch that will rethrow an IOException wrapped with a + * {@see java.lang.RuntimeException}, providing a way to call close() + * without having to do the try-catch everywhere or propagate the IOException. + * + *

Guava has deprecated {@see com.google.common.io.Closeables.closeQuietly()} + * as described in + * {@link https://code.google.com/p/guava-libraries/issues/detail?id=1118}. + * + * @param closeable the Closeable to close + * @throws RuntimeException if an IOException occurs; the IOException is + * wrapped by the RuntimeException + */ + public static void closeNoChecked(final Closeable closeable) { + try { + closeable.close(); + } catch(final IOException e) { + throw new RuntimeException("IOException while closing", e); + } + } +} diff --git a/common/src/main/java/org/apache/drill/common/HistoricalLog.java b/common/src/main/java/org/apache/drill/common/HistoricalLog.java new file mode 100644 index 00000000000..49b7c33f0e1 --- /dev/null +++ b/common/src/main/java/org/apache/drill/common/HistoricalLog.java @@ -0,0 +1,179 @@ +/** + * 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.common; + +import java.util.LinkedList; + +import org.slf4j.Logger; + +/** + * Utility class that can be used to log activity within a class + * for later logging and debugging. Supports recording events and + * recording the stack at the time they occur. + */ +public class HistoricalLog { + private static class Event { + private final String note; // the event text + private final StackTrace stackTrace; // where the event occurred + + public Event(final String note) { + this.note = note; + stackTrace = new StackTrace(); + } + } + + private final LinkedList history = new LinkedList<>(); + private final String idString; // the formatted id string + private Event firstEvent; // the first stack trace recorded + private final int limit; // the limit on the number of events kept + + /** + * Constructor. The format string will be formatted and have its arguments + * substituted at the time this is called. + * + * @param idStringFormat {@link String#format} format string that can be used + * to identify this object in a log. Including some kind of unique identifier + * that can be associated with the object instance is best. + * @param args for the format string, or nothing if none are required + */ + public HistoricalLog(final String idStringFormat, Object... args) { + this(Integer.MAX_VALUE, idStringFormat, args); + } + + /** + * Constructor. The format string will be formatted and have its arguments + * substituted at the time this is called. + * + *

This form supports the specification of a limit that will limit the + * number of historical entries kept (which keeps down the amount of memory + * used). With the limit, the first entry made is always kept (under the + * assumption that this is the creation site of the object, which is usually + * interesting), and then up to the limit number of entries are kept after that. + * Each time a new entry is made, the oldest that is not the first is dropped. + *

+ * + * @param limit the maximum number of historical entries that will be kept, + * not including the first entry made + * @param idStringFormat {@link String#format} format string that can be used + * to identify this object in a log. Including some kind of unique identifier + * that can be associated with the object instance is best. + * @param args for the format string, or nothing if none are required + */ + public HistoricalLog(final int limit, final String idStringFormat, Object... args) { + this.limit = limit; + this.idString = String.format(idStringFormat, args); + } + + /** + * Record an event. Automatically captures the stack trace at the time this is + * called. The format string will be formatted and have its arguments substituted + * at the time this is called. + * + * @param noteFormat {@link String#format} format string that describes the event + * @param args for the format string, or nothing if none are required + */ + public synchronized void recordEvent(final String noteFormat, Object... args) { + final String note = String.format(noteFormat, args); + final Event event = new Event(note); + if (firstEvent == null) { + firstEvent = event; + } + if (history.size() == limit) { + history.removeFirst(); + } + history.add(event); + } + + /** + * Write the history of this object to the given {@link StringBuilder}. The history + * includes the identifying string provided at construction time, and all the recorded + * events with their stack traces. + * + * @param sb {@link StringBuilder} to write to + */ + public void buildHistory(final StringBuilder sb) { + buildHistory(sb, null); + } + + /** + * Write the history of this object to the given {@link StringBuilder}. The history + * includes the identifying string provided at construction time, and all the recorded + * events with their stack traces. + * + * @param sb {@link StringBuilder} to write to + * @param additional an extra string that will be written between the identifying + * information and the history; often used for a current piece of state + */ + public synchronized void buildHistory(final StringBuilder sb, final CharSequence additional) { + sb.append('\n') + .append(idString); + + if (additional != null) { + sb.append('\n') + .append(additional) + .append('\n'); + } + + sb.append(" event log\n"); + + if (firstEvent != null) { + sb.append(" ") + .append(firstEvent.note) + .append('\n'); + firstEvent.stackTrace.writeToBuilder(sb, 4); + + for(final Event event : history) { + if (event == firstEvent) { + continue; + } + + sb.append(" ") + .append(event.note) + .append('\n'); + + event.stackTrace.writeToBuilder(sb, 4); + } + } + } + + /** + * Write the history of this object to the given {@link Logger}. The history + * includes the identifying string provided at construction time, and all the recorded + * events with their stack traces. + * + * @param logger {@link Logger} to write to + * @param additional an extra string that will be written between the identifying + * information and the history; often used for a current piece of state + */ + public void logHistory(final Logger logger, final CharSequence additional) { + final StringBuilder sb = new StringBuilder(); + buildHistory(sb, additional); + logger.debug(sb.toString()); + } + + /** + * Write the history of this object to the given {@link Logger}. The history + * includes the identifying string provided at construction time, and all the recorded + * events with their stack traces. + * + * @param logger {@link Logger} to write to + */ + public void logHistory(final Logger logger) { + logHistory(logger, null); + } +} diff --git a/common/src/main/java/org/apache/drill/common/StackTrace.java b/common/src/main/java/org/apache/drill/common/StackTrace.java index 54068ec6737..de64ed7f8e5 100644 --- a/common/src/main/java/org/apache/drill/common/StackTrace.java +++ b/common/src/main/java/org/apache/drill/common/StackTrace.java @@ -17,9 +17,6 @@ */ package org.apache.drill.common; -import java.io.IOException; -import java.io.StringWriter; -import java.io.Writer; import java.util.Arrays; /** @@ -38,36 +35,35 @@ public StackTrace() { } /** - * Write the stack trace. + * Write the stack trace to a StringBuilder. * - * @param writer where to write it + * @param sb where to write it * @param indent how many spaces to indent each line */ - public void write(final Writer writer, final int indent) { + public void writeToBuilder(final StringBuilder sb, final int indent) { // create the indentation string final char[] indentation = new char[indent]; Arrays.fill(indentation, ' '); - try { - // write the stack trace - for(StackTraceElement ste : stackTraceElements) { - writer.write(indentation); - writer.write(ste.getClassName()); - writer.write('.'); - writer.write(ste.getMethodName()); - writer.write(':'); - writer.write(Integer.toString(ste.getLineNumber())); - writer.write('\n'); - } - } catch(IOException e) { - throw new RuntimeException("couldn't write", e); + // write the stack trace in standard Java format + for(StackTraceElement ste : stackTraceElements) { + sb.append(indentation) + .append("at ") + .append(ste.getClassName()) + .append('.') + .append(ste.getMethodName()) + .append('(') + .append(ste.getFileName()) + .append(':') + .append(Integer.toString(ste.getLineNumber())) + .append(")\n"); } } @Override public String toString() { - final StringWriter sw = new StringWriter(); - write(sw, 0); - return sw.toString(); + final StringBuilder sb = new StringBuilder(); + writeToBuilder(sb, 0); + return sb.toString(); } } diff --git a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java index fe43b4a3005..2038dfb4727 100644 --- a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java +++ b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java @@ -23,8 +23,6 @@ import java.util.Collection; import java.util.List; import java.util.Properties; -import java.util.Queue; -import java.util.concurrent.CopyOnWriteArrayList; import org.apache.drill.common.exceptions.DrillConfigurationException; import org.apache.drill.common.expression.LogicalExpression; @@ -45,8 +43,8 @@ import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; -public final class DrillConfig extends NestedConfig{ -// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillConfig.class); +public final class DrillConfig extends NestedConfig { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillConfig.class); private final ObjectMapper mapper; private final ImmutableList startupArguments; @@ -54,9 +52,6 @@ public final class DrillConfig extends NestedConfig{ @SuppressWarnings("restriction") private static final long MAX_DIRECT_MEMORY = sun.misc.VM.maxDirectMemory(); - @SuppressWarnings("unchecked") - private volatile List> sinkQueues = new CopyOnWriteArrayList>(new Queue[1]); - @VisibleForTesting public DrillConfig(Config config, boolean enableServer) { super(config); @@ -79,11 +74,10 @@ public DrillConfig(Config config, boolean enableServer) { mapper.registerSubtypes(FormatPluginConfigBase.getSubTypes(this)); } - - RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean(); - this.startupArguments = ImmutableList.copyOf(bean.getInputArguments()); + final RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean(); + startupArguments = ImmutableList.copyOf(bean.getInputArguments()); logger.debug("Config object initialized."); - }; + } public List getStartupArguments() { return startupArguments; @@ -155,7 +149,7 @@ private static DrillConfig create(String overrideFileName, Properties overriderP } } - Collection urls = PathScanner.getConfigURLs(); + final Collection urls = PathScanner.getConfigURLs(); logger.debug("Loading configs at the following URLs {}", urls); for (URL url : urls) { fallback = ConfigFactory.parseURL(url).withFallback(fallback); @@ -169,49 +163,41 @@ private static DrillConfig create(String overrideFileName, Properties overriderP return new DrillConfig(effectiveConfig.resolve(), enableServerConfigs); } - public Class getClassAt(String location, Class clazz) throws DrillConfigurationException{ - String className = this.getString(location); + public Class getClassAt(String location, Class clazz) throws DrillConfigurationException { + final String className = getString(location); if (className == null) { - throw new DrillConfigurationException(String.format("No class defined at location '%s'. Expected a definition of the class []", location, clazz.getCanonicalName())); + throw new DrillConfigurationException(String.format( + "No class defined at location '%s'. Expected a definition of the class []", + location, clazz.getCanonicalName())); } - try{ - Class c = Class.forName(className); + + try { + final Class c = Class.forName(className); if (clazz.isAssignableFrom(c)) { - @SuppressWarnings("unchecked") Class t = (Class) c; + @SuppressWarnings("unchecked") + final Class t = (Class) c; return t; - } else { - throw new DrillConfigurationException(String.format("The class [%s] listed at location '%s' should be of type [%s]. It isn't.", className, location, clazz.getCanonicalName())); } + + throw new DrillConfigurationException(String.format("The class [%s] listed at location '%s' should be of type [%s]. It isn't.", className, location, clazz.getCanonicalName())); } catch (Exception ex) { if (ex instanceof DrillConfigurationException) { throw (DrillConfigurationException) ex; } throw new DrillConfigurationException(String.format("Failure while initializing class [%s] described at configuration value '%s'.", className, location), ex); } - } public T getInstanceOf(String location, Class clazz) throws DrillConfigurationException{ - Class c = getClassAt(location, clazz); + final Class c = getClassAt(location, clazz); try { - T t = c.newInstance(); + final T t = c.newInstance(); return t; } catch (Exception ex) { throw new DrillConfigurationException(String.format("Failure while instantiating class [%s] located at '%s.", clazz.getCanonicalName(), location), ex); } } - public void setSinkQueues(int number, Queue queue) { - sinkQueues.set(number, queue); - } - - public Queue getQueue(int number) { - if (sinkQueues.size() <= number || number < 0 || sinkQueues == null) { - throw new IllegalArgumentException(String.format("Queue %d is not available.", number)); - } - return sinkQueues.get(number); - } - public ObjectMapper getMapper() { return mapper; } @@ -221,9 +207,9 @@ public String toString() { return this.root().render(); } - public static void main(String[] args) throws Exception { + public static void main(String[] args) { //"-XX:MaxDirectMemorySize" - DrillConfig config = DrillConfig.create(); + @SuppressWarnings("unused") final DrillConfig config = DrillConfig.create(); } public static long getMaxDirectMemory() { diff --git a/common/src/main/java/org/apache/drill/common/config/NestedConfig.java b/common/src/main/java/org/apache/drill/common/config/NestedConfig.java index 3fd885f0ed9..60fe013dc2e 100644 --- a/common/src/main/java/org/apache/drill/common/config/NestedConfig.java +++ b/common/src/main/java/org/apache/drill/common/config/NestedConfig.java @@ -30,7 +30,7 @@ import com.typesafe.config.ConfigValue; abstract class NestedConfig implements Config { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NestedConfig.class); + // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NestedConfig.class); private final Config c; @@ -38,164 +38,203 @@ abstract class NestedConfig implements Config { this.c = c; } + @Override public ConfigObject root() { return c.root(); } + @Override public ConfigOrigin origin() { return c.origin(); } + @Override public Config withFallback(ConfigMergeable other) { return c.withFallback(other); } + @Override public Config resolve() { return c.resolve(); } + @Override public Config resolve(ConfigResolveOptions options) { return c.resolve(options); } + @Override public void checkValid(Config reference, String... restrictToPaths) { c.checkValid(reference, restrictToPaths); } + @Override public boolean hasPath(String path) { return c.hasPath(path); } + @Override public boolean isEmpty() { return c.isEmpty(); } + @Override public Set> entrySet() { return c.entrySet(); } + @Override public boolean getBoolean(String path) { return c.getBoolean(path); } + @Override public Number getNumber(String path) { return c.getNumber(path); } + @Override public int getInt(String path) { return c.getInt(path); } + @Override public long getLong(String path) { return c.getLong(path); } + @Override public double getDouble(String path) { return c.getDouble(path); } + @Override public String getString(String path) { return c.getString(path); } + @Override public ConfigObject getObject(String path) { return c.getObject(path); } + @Override public Config getConfig(String path) { return c.getConfig(path); } + @Override public Object getAnyRef(String path) { return c.getAnyRef(path); } + @Override public ConfigValue getValue(String path) { return c.getValue(path); } + @Override public Long getBytes(String path) { return c.getBytes(path); } + @Override public Long getMilliseconds(String path) { return c.getMilliseconds(path); } + @Override public Long getNanoseconds(String path) { return c.getNanoseconds(path); } + @Override public ConfigList getList(String path) { return c.getList(path); } + @Override public List getBooleanList(String path) { return c.getBooleanList(path); } + @Override public List getNumberList(String path) { return c.getNumberList(path); } + @Override public List getIntList(String path) { return c.getIntList(path); } + @Override public List getLongList(String path) { return c.getLongList(path); } + @Override public List getDoubleList(String path) { return c.getDoubleList(path); } + @Override public List getStringList(String path) { return c.getStringList(path); } + @Override public List getObjectList(String path) { return c.getObjectList(path); } + @Override public List getConfigList(String path) { return c.getConfigList(path); } + @Override public List getAnyRefList(String path) { return c.getAnyRefList(path); } + @Override public List getBytesList(String path) { return c.getBytesList(path); } + @Override public List getMillisecondsList(String path) { return c.getMillisecondsList(path); } + @Override public List getNanosecondsList(String path) { return c.getNanosecondsList(path); } + @Override public Config withOnlyPath(String path) { return c.withOnlyPath(path); } + @Override public Config withoutPath(String path) { return c.withoutPath(path); } + @Override public Config atPath(String path) { return c.atPath(path); } + @Override public Config atKey(String key) { return c.atKey(key); } + @Override public Config withValue(String path, ConfigValue value) { return c.withValue(path, value); } - } diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseRecordReader.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseRecordReader.java index 9458db26e8c..ba105920b80 100644 --- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseRecordReader.java +++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseRecordReader.java @@ -31,9 +31,9 @@ import org.apache.drill.common.expression.PathSegment.NameSegment; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.exec.exception.SchemaChangeException; -import org.apache.drill.exec.memory.OutOfMemoryException; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.ops.OperatorContext; +import org.apache.drill.exec.ops.OperatorStats; import org.apache.drill.exec.physical.impl.OutputMutator; import org.apache.drill.exec.record.MaterializedField; import org.apache.drill.exec.store.AbstractRecordReader; @@ -75,7 +75,7 @@ public class HBaseRecordReader extends AbstractRecordReader implements DrillHBas private boolean rowKeyOnly; public HBaseRecordReader(Configuration conf, HBaseSubScan.HBaseSubScanSpec subScanSpec, - List projectedColumns, FragmentContext context) throws OutOfMemoryException { + List projectedColumns, FragmentContext context) { hbaseConf = conf; hbaseTableName = Preconditions.checkNotNull(subScanSpec, "HBase reader needs a sub-scan spec").getTableName(); hbaseScan = new Scan(subScanSpec.getStartRow(), subScanSpec.getStopRow()); @@ -169,15 +169,16 @@ public int next() { done: for (; rowCount < TARGET_RECORD_COUNT; rowCount++) { Result result = null; + final OperatorStats operatorStats = operatorContext == null ? null : operatorContext.getStats(); try { - if (operatorContext != null) { - operatorContext.getStats().startWait(); + if (operatorStats != null) { + operatorStats.startWait(); } try { result = resultScanner.next(); } finally { - if (operatorContext != null) { - operatorContext.getStats().stopWait(); + if (operatorStats != null) { + operatorStats.stopWait(); } } } catch (IOException e) { @@ -193,20 +194,20 @@ public int next() { rowKeyVector.getMutator().setSafe(rowCount, cells[0].getRowArray(), cells[0].getRowOffset(), cells[0].getRowLength()); } if (!rowKeyOnly) { - for (Cell cell : cells) { - int familyOffset = cell.getFamilyOffset(); - int familyLength = cell.getFamilyLength(); - byte[] familyArray = cell.getFamilyArray(); - MapVector mv = getOrCreateFamilyVector(new String(familyArray, familyOffset, familyLength), true); + for (final Cell cell : cells) { + final int familyOffset = cell.getFamilyOffset(); + final int familyLength = cell.getFamilyLength(); + final byte[] familyArray = cell.getFamilyArray(); + final MapVector mv = getOrCreateFamilyVector(new String(familyArray, familyOffset, familyLength), true); - int qualifierOffset = cell.getQualifierOffset(); - int qualifierLength = cell.getQualifierLength(); - byte[] qualifierArray = cell.getQualifierArray(); - NullableVarBinaryVector v = getOrCreateColumnVector(mv, new String(qualifierArray, qualifierOffset, qualifierLength)); + final int qualifierOffset = cell.getQualifierOffset(); + final int qualifierLength = cell.getQualifierLength(); + final byte[] qualifierArray = cell.getQualifierArray(); + final NullableVarBinaryVector v = getOrCreateColumnVector(mv, new String(qualifierArray, qualifierOffset, qualifierLength)); - int valueOffset = cell.getValueOffset(); - int valueLength = cell.getValueLength(); - byte[] valueArray = cell.getValueArray(); + final int valueOffset = cell.getValueOffset(); + final int valueLength = cell.getValueLength(); + final byte[] valueArray = cell.getValueArray(); v.getMutator().setSafe(rowCount, valueArray, valueOffset, valueLength); } } @@ -246,7 +247,7 @@ private NullableVarBinaryVector getOrCreateColumnVector(MapVector mv, String qua } @Override - public void cleanup() { + public void close() { try { if (resultScanner != null) { resultScanner.close(); @@ -267,5 +268,4 @@ private void setOutputRowCount(int count) { rowKeyVector.getMutator().setValueCount(count); } } - } 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 088fb74208d..cec72959c2f 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 @@ -44,7 +44,6 @@ import org.apache.drill.exec.physical.impl.OutputMutator; import org.apache.drill.exec.planner.physical.PlannerSettings; 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; @@ -94,8 +93,7 @@ import com.google.common.collect.Lists; public class HiveRecordReader extends AbstractRecordReader { - - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveRecordReader.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveRecordReader.class); protected Table table; protected Partition partition; @@ -166,7 +164,7 @@ private void init() throws ExecutionSetupException { String inputFormatName = (partition == null) ? table.getSd().getInputFormat() : partition.getSd().getInputFormat(); try { format = (InputFormat) Class.forName(inputFormatName).getConstructor().newInstance(); - Class c = Class.forName(sLib); + Class c = Class.forName(sLib); serde = (SerDe) c.getConstructor().newInstance(); serde.initialize(job, properties); } catch (ReflectiveOperationException | SerDeException e) { @@ -317,7 +315,6 @@ public int next() { } private boolean readHiveRecordAndInsertIntoRecordBatch(Object deSerializedValue, int outputRecordIndex) { - boolean success; for (int i = 0; i < selectedColumnNames.size(); i++) { String columnName = selectedColumnNames.get(i); Object hiveValue = sInspector.getStructFieldData(deSerializedValue, sInspector.getStructFieldRef(columnName)); @@ -342,7 +339,7 @@ private void setValueCountAndPopulatePartitionVectors(int recordCount) { } @Override - public void cleanup() { + public void close() { try { if (reader != null) { reader.close(); diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java index 0ac519fc7a4..c2ddff257f3 100644 --- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java +++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java @@ -52,7 +52,7 @@ import com.mongodb.client.MongoDatabase; public class MongoRecordReader extends AbstractRecordReader { - static final Logger logger = LoggerFactory.getLogger(MongoRecordReader.class); + private static final Logger logger = LoggerFactory.getLogger(MongoRecordReader.class); private MongoCollection collection; private MongoCursor cursor; @@ -187,7 +187,7 @@ public int next() { } @Override - public void cleanup() { + public void close() { } diff --git a/exec/java-exec/src/main/codegen/templates/AbstractFieldWriter.java b/exec/java-exec/src/main/codegen/templates/AbstractFieldWriter.java index 1b5dad1d1bf..9b673048e56 100644 --- a/exec/java-exec/src/main/codegen/templates/AbstractFieldWriter.java +++ b/exec/java-exec/src/main/codegen/templates/AbstractFieldWriter.java @@ -26,58 +26,66 @@ <#include "/@includes/vv_imports.ftl" /> -/* This class is generated using freemarker and the AbstractFieldWriter.java template */ +/* + * This class is generated using freemarker and the ${.template_name} template. + */ @SuppressWarnings("unused") -abstract class AbstractFieldWriter extends AbstractBaseWriter implements FieldWriter{ - - AbstractFieldWriter(FieldWriter parent){ +abstract class AbstractFieldWriter extends AbstractBaseWriter implements FieldWriter { + AbstractFieldWriter(FieldWriter parent) { super(parent); } - - public void start(){ + + @Override + public void start() { throw new IllegalStateException(String.format("You tried to start when you are using a ValueWriter of type %s.", this.getClass().getSimpleName())); } - - public void end(){ + + @Override + public void end() { throw new IllegalStateException(String.format("You tried to end when you are using a ValueWriter of type %s.", this.getClass().getSimpleName())); } - + <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first /> <#assign fields = minor.fields!type.fields /> - public void write(${name}Holder holder){ + @Override + public void write(${name}Holder holder) { fail("${name}"); } - - public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, ){ + + public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, ) { fail("${name}"); } - + - public void writeNull(){ + public void writeNull() { fail("${name}"); } - - public MapWriter map(){ + + @Override + public MapWriter map() { fail("Map"); return null; } - - public ListWriter list(){ + + @Override + public ListWriter list() { fail("List"); return null; } - - public MapWriter map(String name){ + + @Override + public MapWriter map(String name) { fail("Map"); return null; } - - public ListWriter list(String name){ + + @Override + public ListWriter list(String name) { fail("List"); return null; } - + <#list vv.types as type><#list type.minor as minor> <#assign lowerName = minor.class?uncap_first /> <#if lowerName == "int" ><#assign lowerName = "integer" /> @@ -89,32 +97,30 @@ public ListWriter list(String name){ return null; } + + @Override public ${capName}Writer ${lowerName}(String name) { fail("${capName}"); return null; } - public ${capName}Writer ${lowerName}(){ + + @Override + public ${capName}Writer ${lowerName}() { fail("${capName}"); return null; } - - + - - - public void copyReader(FieldReader reader){ + + public void copyReader(FieldReader reader) { fail("Copy FieldReader"); } - public void copyReaderToField(String name, FieldReader reader){ + + public void copyReaderToField(String name, FieldReader reader) { fail("Copy FieldReader to STring"); } - - private void fail(String name){ + + private void fail(String name) { throw new IllegalArgumentException(String.format("You tried to write a %s type when you are using a ValueWriter of type %s.", name, this.getClass().getSimpleName())); } - - } - - - diff --git a/exec/java-exec/src/main/codegen/templates/BaseWriter.java b/exec/java-exec/src/main/codegen/templates/BaseWriter.java index ada410d5d2b..76978801fb3 100644 --- a/exec/java-exec/src/main/codegen/templates/BaseWriter.java +++ b/exec/java-exec/src/main/codegen/templates/BaseWriter.java @@ -26,13 +26,15 @@ <#include "/@includes/vv_imports.ftl" /> -/* This class is generated using freemarker and the BaseWriter.java template */ +/* + * File generated from ${.template_name} using FreeMarker. + */ @SuppressWarnings("unused") -public interface BaseWriter extends Positionable{ + public interface BaseWriter extends AutoCloseable, Positionable { FieldWriter getParent(); int getValueCapacity(); - public interface MapWriter extends BaseWriter{ + public interface MapWriter extends BaseWriter { MaterializedField getField(); @@ -46,21 +48,21 @@ public interface MapWriter extends BaseWriter{ ${capName}Writer ${lowerName}(String name); - + void copyReaderToField(String name, FieldReader reader); MapWriter map(String name); ListWriter list(String name); void start(); void end(); } - - public interface ListWriter extends BaseWriter{ + + public interface ListWriter extends BaseWriter { void start(); void end(); MapWriter map(); ListWriter list(); void copyReader(FieldReader reader); - + <#list vv.types as type><#list type.minor as minor> <#assign lowerName = minor.class?uncap_first /> <#if lowerName == "int" ><#assign lowerName = "integer" /> @@ -69,20 +71,19 @@ public interface ListWriter extends BaseWriter{ ${capName}Writer ${lowerName}(); } - - public interface ScalarWriter extends + + public interface ScalarWriter extends <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first /> ${name}Writer, BaseWriter {} - - public interface ComplexWriter{ + + public interface ComplexWriter { void allocate(); void clear(); void copyReader(FieldReader reader); MapWriter rootAsMap(); ListWriter rootAsList(); - - public void setPosition(int index); - public void setValueCount(int count); - public void reset(); + + void setPosition(int index); + void setValueCount(int count); + void reset(); } } - diff --git a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java b/exec/java-exec/src/main/codegen/templates/ComplexWriters.java index 980f9ac6842..71a7f9a0ccb 100644 --- a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java +++ b/exec/java-exec/src/main/codegen/templates/ComplexWriters.java @@ -40,92 +40,103 @@ <#include "/@includes/vv_imports.ftl" /> -/* This class is generated using freemarker and the ComplexWriters.java template */ +/* + * This class is generated using FreeMarker on the ${.template_name} template. + */ @SuppressWarnings("unused") public class ${eName}WriterImpl extends AbstractFieldWriter { - + private final ${name}Vector.Mutator mutator; final ${name}Vector vector; - - public ${eName}WriterImpl(${name}Vector vector, AbstractFieldWriter parent){ + + public ${eName}WriterImpl(${name}Vector vector, AbstractFieldWriter parent) { super(parent); this.mutator = vector.getMutator(); this.vector = vector; } - public MaterializedField getField(){ + @Override + public MaterializedField getField() { return vector.getField(); } + @Override public int getValueCapacity() { return vector.getValueCapacity(); } - public void allocate(){ + @Override + public void allocate() { vector.allocateNew(); } - - public void clear(){ + + @Override + public void close() { + vector.close(); + } + + @Override + public void clear() { vector.clear(); } - - protected int idx(){ + + @Override + protected int idx() { return super.idx(); } - + <#if mode == "Repeated"> - public void write(${minor.class?cap_first}Holder h){ + public void write(${minor.class?cap_first}Holder h) { mutator.addSafe(idx(), h); vector.getMutator().setValueCount(idx()+1); } - - public void write(Nullable${minor.class?cap_first}Holder h){ + + public void write(Nullable${minor.class?cap_first}Holder h) { mutator.addSafe(idx(), h); vector.getMutator().setValueCount(idx()+1); } <#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>, ){ + public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, ) { mutator.addSafe(idx(), <#list fields as field>${field.name}<#if field_has_next>, ); vector.getMutator().setValueCount(idx()+1); } - - public void setPosition(int idx){ + + public void setPosition(int idx) { super.setPosition(idx); mutator.startNewValue(idx); } - - + + <#else> - - public void write(${minor.class}Holder h){ + + public void write(${minor.class}Holder h) { mutator.setSafe(idx(), h); vector.getMutator().setValueCount(idx()+1); } - - public void write(Nullable${minor.class}Holder h){ + + public void write(Nullable${minor.class}Holder h) { mutator.setSafe(idx(), h); vector.getMutator().setValueCount(idx()+1); } <#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>, ){ + public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, ) { mutator.setSafe(idx(), <#if mode == "Nullable">1, <#list fields as field>${field.name}<#if field_has_next>, ); vector.getMutator().setValueCount(idx()+1); } <#if mode == "Nullable"> - public void writeNull(){ + + public void writeNull() { mutator.setNull(idx()); vector.getMutator().setValueCount(idx()+1); } - - } <@pp.changeOutputFile name="/org/apache/drill/exec/vector/complex/writer/${eName}Writer.java" /> @@ -135,18 +146,14 @@ public void writeNull(){ <#include "/@includes/vv_imports.ftl" /> @SuppressWarnings("unused") -public interface ${eName}Writer extends BaseWriter{ +public interface ${eName}Writer extends BaseWriter { public void write(${minor.class}Holder h); - + <#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>, ); } - - - - diff --git a/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java b/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java index cadcfd92bad..5ff2ef00984 100644 --- a/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java +++ b/exec/java-exec/src/main/codegen/templates/FixedValueVectors.java @@ -38,10 +38,9 @@ * The width of each element is ${type.width} byte(s) * The equivalent Java primitive is '${minor.javaType!type.javaType}' * - * Source code generated using FreeMarker template ${.template_name} + * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker. */ -@SuppressWarnings("unused") -public final class ${minor.class}Vector extends BaseDataValueVector implements FixedWidthVector{ +public final class ${minor.class}Vector extends BaseDataValueVector implements FixedWidthVector { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class); private final FieldReader reader = new ${minor.class}ReaderImpl(${minor.class}Vector.this); @@ -56,19 +55,22 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F } @Override - public FieldReader getReader(){ + public FieldReader getReader() { return reader; } - public int getValueCapacity(){ - return (int) (data.capacity() *1.0 / ${type.width}); + @Override + public int getValueCapacity() { + return (int) (data.capacity() * 1.0 / ${type.width}); } - public Accessor getAccessor(){ + @Override + public Accessor getAccessor() { return accessor; } - public Mutator getMutator(){ + @Override + public Mutator getMutator() { return mutator; } @@ -81,12 +83,14 @@ public void setInitialCapacity(final int valueCount) { allocationSizeInBytes = (int)size; } + @Override public void allocateNew() { - if(!allocateNewSafe()){ + if(!allocateNewSafe()) { throw new OutOfMemoryRuntimeException("Failure while allocating buffer."); } } + @Override public boolean allocateNewSafe() { long curAllocationSize = allocationSizeInBytes; if (allocationMonitor > 10) { @@ -97,7 +101,7 @@ public boolean allocateNewSafe() { allocationMonitor = 0; } - try{ + try { allocateBytes(curAllocationSize); } catch (DrillRuntimeException ex) { return false; @@ -114,6 +118,7 @@ public boolean allocateNewSafe() { * @param valueCount * @throws org.apache.drill.exec.memory.OutOfMemoryRuntimeException if it can't allocate the new buffer */ + @Override public void allocateNew(final int valueCount) { allocateBytes(valueCount * ${type.width}); } @@ -142,25 +147,33 @@ public void reAlloc() { } logger.debug("Reallocating vector [{}]. # of bytes: [{}] -> [{}]", field, allocationSizeInBytes, newAllocationSize); - final DrillBuf newBuf = allocator.buffer((int)newAllocationSize); + final DrillBuf newBuf = allocator.buffer((int) newAllocationSize); newBuf.setBytes(0, data, 0, data.capacity()); - newBuf.setZero(newBuf.capacity() / 2, newBuf.capacity() / 2); + final int halfNewCapacity = newBuf.capacity() / 2; + newBuf.setZero(halfNewCapacity, halfNewCapacity); newBuf.writerIndex(data.writerIndex()); - data.release(); + data.release(1); data = newBuf; - allocationSizeInBytes = (int)newAllocationSize; + allocationSizeInBytes = (int) newAllocationSize; } + /** + * {@inheritDoc} + */ + @Override public void zeroVector() { data.setZero(0, data.capacity()); } @Override - public int load(int valueCount, DrillBuf buf){ + public int load(int valueCount, DrillBuf buf) { clear(); - int len = valueCount * ${type.width}; + final int len = valueCount * ${type.width}; + if (data != null) { + data.release(1); + } data = buf.slice(0, len); - data.retain(); + data.retain(1); data.writerIndex(len); return len; } @@ -168,58 +181,64 @@ public int load(int valueCount, DrillBuf buf){ @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); + final 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(){ + @Override + public TransferPair getTransferPair() { return new TransferImpl(getField()); } + + @Override public TransferPair getTransferPair(FieldReference ref){ return new TransferImpl(getField().withPath(ref)); } + @Override public TransferPair makeTransferPair(ValueVector to) { return new TransferImpl((${minor.class}Vector) to); } - public void transferTo(${minor.class}Vector target){ + public void transferTo(${minor.class}Vector target) { target.clear(); target.data = data; - target.data.retain(); + target.data.retain(1); target.data.writerIndex(data.writerIndex()); clear(); } public void splitAndTransferTo(int startIndex, int length, ${minor.class}Vector target) { - int currentWriterIndex = data.writerIndex(); - int startPoint = startIndex * ${type.width}; - int sliceLength = length * ${type.width}; + final int startPoint = startIndex * ${type.width}; + final int sliceLength = length * ${type.width}; target.clear(); - target.data = this.data.slice(startPoint, sliceLength); + target.data = data.slice(startPoint, sliceLength); + target.data.retain(1); target.data.writerIndex(sliceLength); - target.data.retain(); } - private class TransferImpl implements TransferPair{ - ${minor.class}Vector to; + private class TransferImpl implements TransferPair { + private ${minor.class}Vector to; - public TransferImpl(MaterializedField field){ - this.to = new ${minor.class}Vector(field, allocator); + public TransferImpl(MaterializedField field) { + to = new ${minor.class}Vector(field, allocator); } public TransferImpl(${minor.class}Vector to) { this.to = to; } - public ${minor.class}Vector getTo(){ + @Override + public ${minor.class}Vector getTo() { return to; } - public void transfer(){ + @Override + public void transfer() { transferTo(to); } + @Override public void splitAndTransfer(int startIndex, int length) { splitAndTransferTo(startIndex, length, to); } @@ -230,7 +249,7 @@ public void copyValueSafe(int fromIndex, int toIndex) { } } - public void copyFrom(int fromIndex, int thisIndex, ${minor.class}Vector from){ + 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 --> @@ -240,7 +259,7 @@ public void copyFrom(int fromIndex, int thisIndex, ${minor.class}Vector from){ <#-- type.width --> } - public void copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from){ + public void copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from) { while(thisIndex >= getValueCapacity()) { reAlloc(); } @@ -259,12 +278,13 @@ private void incrementAllocationMonitor() { } public final class Accessor extends BaseDataValueVector.BaseAccessor { - + @Override public int getValueCount() { return data.writerIndex() / ${type.width}; } - public boolean isNull(int index){ + @Override + public boolean isNull(int index) { return false; } @@ -275,16 +295,16 @@ public boolean isNull(int index){ } <#if (minor.class == "Interval")> - public void get(int index, ${minor.class}Holder holder){ + public void get(int index, ${minor.class}Holder holder) { - int offsetIndex = index * ${type.width}; + final int offsetIndex = index * ${type.width}; holder.months = data.getInt(offsetIndex); holder.days = data.getInt(offsetIndex + ${minor.daysOffset}); holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset}); } - public void get(int index, Nullable${minor.class}Holder holder){ - int offsetIndex = index * ${type.width}; + public void get(int index, Nullable${minor.class}Holder holder) { + final int offsetIndex = index * ${type.width}; holder.isSet = 1; holder.months = data.getInt(offsetIndex); holder.days = data.getInt(offsetIndex + ${minor.daysOffset}); @@ -293,37 +313,37 @@ public void get(int index, Nullable${minor.class}Holder holder){ @Override public ${friendlyType} getObject(int index) { - int offsetIndex = index * ${type.width}; - int months = data.getInt(offsetIndex); - int days = data.getInt(offsetIndex + ${minor.daysOffset}); - int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset}); - Period p = new Period(); + final int offsetIndex = index * ${type.width}; + final int months = data.getInt(offsetIndex); + final int days = data.getInt(offsetIndex + ${minor.daysOffset}); + final int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset}); + final Period p = new Period(); return p.plusMonths(months).plusDays(days).plusMillis(millis); } public StringBuilder getAsStringBuilder(int index) { - int offsetIndex = index * ${type.width}; + final int offsetIndex = index * ${type.width}; int months = data.getInt(offsetIndex); - int days = data.getInt(offsetIndex + ${minor.daysOffset}); + final int days = data.getInt(offsetIndex + ${minor.daysOffset}); int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset}); - int years = (months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); + final int years = (months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); months = (months % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); - int hours = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis); + final int hours = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis); millis = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis); - int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis); + final int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis); millis = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis); - long seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis); + final long seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis); millis = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis); - String yearString = (Math.abs(years) == 1) ? " year " : " years "; - String monthString = (Math.abs(months) == 1) ? " month " : " months "; - String dayString = (Math.abs(days) == 1) ? " day " : " days "; + final String yearString = (Math.abs(years) == 1) ? " year " : " years "; + final String monthString = (Math.abs(months) == 1) ? " month " : " months "; + final String dayString = (Math.abs(days) == 1) ? " day " : " days "; return(new StringBuilder(). @@ -337,15 +357,15 @@ public StringBuilder getAsStringBuilder(int index) { } <#elseif (minor.class == "IntervalDay")> - public void get(int index, ${minor.class}Holder holder){ + public void get(int index, ${minor.class}Holder holder) { - int offsetIndex = index * ${type.width}; + final int offsetIndex = index * ${type.width}; holder.days = data.getInt(offsetIndex); holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset}); } - public void get(int index, Nullable${minor.class}Holder holder){ - int offsetIndex = index * ${type.width}; + public void get(int index, Nullable${minor.class}Holder holder) { + final int offsetIndex = index * ${type.width}; holder.isSet = 1; holder.days = data.getInt(offsetIndex); holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset}); @@ -353,30 +373,30 @@ public void get(int index, Nullable${minor.class}Holder holder){ @Override public ${friendlyType} getObject(int index) { - int offsetIndex = index * ${type.width}; - int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset}); - int days = data.getInt(offsetIndex); - Period p = new Period(); + final int offsetIndex = index * ${type.width}; + final int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset}); + final int days = data.getInt(offsetIndex); + final Period p = new Period(); return p.plusDays(days).plusMillis(millis); } public StringBuilder getAsStringBuilder(int index) { - int offsetIndex = index * ${type.width}; + final int offsetIndex = index * ${type.width}; int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset}); - int days = data.getInt(offsetIndex); + final int days = data.getInt(offsetIndex); - int hours = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis); + final int hours = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis); millis = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis); - int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis); + final int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis); millis = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis); - int seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis); + final int seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis); millis = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis); - String dayString = (Math.abs(days) == 1) ? " day " : " days "; + final String dayString = (Math.abs(days) == 1) ? " day " : " days "; return(new StringBuilder(). append(days).append(dayString). @@ -414,12 +434,12 @@ public void get(int index, Nullable${minor.class}Holder holder) { } <#else> - public void get(int index, ${minor.class}Holder holder){ + public void get(int index, ${minor.class}Holder holder) { holder.buffer = data; holder.start = index * ${type.width}; } - public void get(int index, Nullable${minor.class}Holder holder){ + public void get(int index, Nullable${minor.class}Holder holder) { holder.isSet = 1; holder.buffer = data; holder.start = index * ${type.width}; @@ -445,6 +465,7 @@ public long getTwoAsLong(int index) { <#if minor.class == "Date"> + @Override 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()); @@ -452,6 +473,7 @@ public long getTwoAsLong(int index) { } <#elseif minor.class == "TimeStamp"> + @Override 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()); @@ -459,13 +481,14 @@ public long getTwoAsLong(int index) { } <#elseif minor.class == "IntervalYear"> + @Override public ${friendlyType} getObject(int index) { - int value = get(index); + final int value = get(index); - int years = (value / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); - int months = (value % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); - Period p = new Period(); + final int years = (value / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); + final int months = (value % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); + final Period p = new Period(); return p.plusYears(years).plusMonths(months); } @@ -473,11 +496,11 @@ public StringBuilder getAsStringBuilder(int index) { int months = data.getInt(index); - int years = (months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); + final int years = (months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); months = (months % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths); - String yearString = (Math.abs(years) == 1) ? " year " : " years "; - String monthString = (Math.abs(months) == 1) ? " month " : " months "; + final String yearString = (Math.abs(years) == 1) ? " year " : " years "; + final String monthString = (Math.abs(months) == 1) ? " month " : " months "; return(new StringBuilder(). append(years).append(yearString). @@ -493,17 +516,16 @@ public DateTime getObject(int index) { return time; } - - <#elseif minor.class == "Decimal9" || minor.class == "Decimal18"> @Override public ${friendlyType} getObject(int index) { - BigInteger value = BigInteger.valueOf(((${type.boxedType})get(index)).${type.javaType}Value()); + final BigInteger value = BigInteger.valueOf(((${type.boxedType})get(index)).${type.javaType}Value()); return new BigDecimal(value, getField().getScale()); } <#else> + @Override public ${friendlyType} getObject(int index) { return get(index); } @@ -512,7 +534,7 @@ public DateTime getObject(int index) { } - public void get(int index, ${minor.class}Holder holder){ + public void get(int index, ${minor.class}Holder holder) { <#if minor.class.startsWith("Decimal")> holder.scale = getField().getScale(); holder.precision = getField().getPrecision(); @@ -521,7 +543,7 @@ public void get(int index, ${minor.class}Holder holder){ holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}); } - public void get(int index, Nullable${minor.class}Holder holder){ + 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}); } @@ -541,7 +563,7 @@ public void get(int index, Nullable${minor.class}Holder holder){ */ public final class Mutator extends BaseDataValueVector.BaseMutator { - private Mutator(){}; + private Mutator() {}; /** * Set the element at the given index to the given value. Note that widths smaller than * 32 bits are handled by the DrillBuf interface. @@ -562,122 +584,122 @@ public void setSafe(int index, <#if (type.width > 4)>${minor.javaType!type.javaT } <#if (minor.class == "Interval")> - public void set(int index, int months, int days, int milliseconds){ - int offsetIndex = index * ${type.width}; + public void set(int index, int months, int days, int milliseconds) { + final int offsetIndex = index * ${type.width}; data.setInt(offsetIndex, months); data.setInt((offsetIndex + ${minor.daysOffset}), days); data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds); } - protected void set(int index, ${minor.class}Holder holder){ + 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){ + protected void set(int index, Nullable${minor.class}Holder holder) { set(index, holder.months, holder.days, holder.milliseconds); } - public void setSafe(int index, int months, int days, int milliseconds){ + public void setSafe(int index, int months, int days, int milliseconds) { while(index >= getValueCapacity()) { reAlloc(); } set(index, months, days, milliseconds); } - public void setSafe(int index, Nullable${minor.class}Holder holder){ + 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){ + 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}; + public void set(int index, int days, int milliseconds) { + final int offsetIndex = index * ${type.width}; data.setInt(offsetIndex, days); data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds); } - protected void set(int index, ${minor.class}Holder holder){ + protected void set(int index, ${minor.class}Holder holder) { set(index, holder.days, holder.milliseconds); } - protected void set(int index, Nullable${minor.class}Holder holder){ + protected void set(int index, Nullable${minor.class}Holder holder) { set(index, holder.days, holder.milliseconds); } - public void setSafe(int index, int days, int milliseconds){ + public void setSafe(int index, int days, int milliseconds) { while(index >= getValueCapacity()) { reAlloc(); } set(index, days, milliseconds); } - public void setSafe(int index, ${minor.class}Holder holder){ + public void setSafe(int index, ${minor.class}Holder holder) { setSafe(index, holder.days, holder.milliseconds); } - public void setSafe(int index, Nullable${minor.class}Holder holder){ + public void setSafe(int index, Nullable${minor.class}Holder holder) { setSafe(index, holder.days, holder.milliseconds); } <#elseif (minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse") || (minor.class == "Decimal28Dense") || (minor.class == "Decimal38Dense")> - public void set(int index, ${minor.class}Holder holder){ + public void set(int index, ${minor.class}Holder holder) { set(index, holder.start, holder.buffer); } - void set(int index, Nullable${minor.class}Holder holder){ + 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){ + 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){ + public void setSafe(int index, ${minor.class}Holder holder) { setSafe(index, holder.start, holder.buffer); } - public void setSafe(int index, int start, DrillBuf 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){ + 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){ + 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){ + public void set(int index, Nullable${minor.class}Holder holder) { set(index, holder.start, holder.buffer); } - public void set(int index, int start, DrillBuf buffer){ + 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){ + 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){ + 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){ + public void setSafe(int index, int start, DrillBuf buffer) { while(index >= getValueCapacity()) { reAlloc(); } set(index, holder); } - public void set(int index, Nullable${minor.class}Holder holder){ + public void set(int index, Nullable${minor.class}Holder holder) { data.setBytes(index * ${type.width}, holder.buffer, holder.start, ${type.width}); } @@ -686,17 +708,15 @@ public void set(int index, Nullable${minor.class}Holder holder){ public void generateTestData(int count) { setValueCount(count); boolean even = true; - 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++){ + final int valueCount = getAccessor().getValueCount(); + for(int i = 0; i < valueCount; i++, even = !even) { + final byte b = even ? Byte.MIN_VALUE : Byte.MAX_VALUE; + for(int w = 0; w < ${type.width}; w++) { data.setByte(i + w, b); } } } - - - <#else> <#-- type.width <= 8 --> public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int value) { data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, value); @@ -709,22 +729,22 @@ public void setSafe(int index, <#if (type.width >= 4)>${minor.javaType!type.java set(index, value); } - protected void set(int index, ${minor.class}Holder holder){ + protected void set(int index, ${minor.class}Holder holder) { data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, holder.value); } - public void setSafe(int index, ${minor.class}Holder holder){ + public void setSafe(int index, ${minor.class}Holder holder) { while(index >= getValueCapacity()) { reAlloc(); } set(index, holder); } - protected void set(int index, Nullable${minor.class}Holder holder){ + protected void set(int index, Nullable${minor.class}Holder holder) { data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, holder.value); } - public void setSafe(int index, Nullable${minor.class}Holder holder){ + public void setSafe(int index, Nullable${minor.class}Holder holder) { while(index >= getValueCapacity()) { reAlloc(); } @@ -735,35 +755,27 @@ public void setSafe(int index, Nullable${minor.class}Holder holder){ public void generateTestData(int size) { setValueCount(size); boolean even = true; - for(int i =0; i < getAccessor().getValueCount(); i++, even = !even){ - if(even){ - set(i, ${minor.boxedType!type.boxedType}.MIN_VALUE); - }else{ - set(i, ${minor.boxedType!type.boxedType}.MAX_VALUE); - } + final int valueCount = getAccessor().getValueCount(); + for(int i = 0; i < valueCount; i++, even = !even) { + set(i, even ? ${minor.boxedType!type.boxedType}.MIN_VALUE : ${minor.boxedType!type.boxedType}.MAX_VALUE); } } - public void generateTestDataAlt(int size) { setValueCount(size); boolean even = true; - for(int i =0; i < getAccessor().getValueCount(); i++, even = !even){ - if(even){ - set(i, (${(minor.javaType!type.javaType)}) 1); - }else{ - set(i, (${(minor.javaType!type.javaType)}) 0); - } + final int valueCount = getAccessor().getValueCount(); + for(int i = 0; i < valueCount; i++, even = !even) { + set(i, (${(minor.javaType!type.javaType)}) (even ? 1 : 0)); } } <#-- type.width --> - - + @Override public void setValueCount(int valueCount) { - int currentValueCapacity = getValueCapacity(); - int idx = (${type.width} * valueCount); + final int currentValueCapacity = getValueCapacity(); + final int idx = (${type.width} * valueCount); while(valueCount > getValueCapacity()) { reAlloc(); } @@ -775,11 +787,6 @@ public void setValueCount(int valueCount) { VectorTrimmer.trim(data, idx); data.writerIndex(valueCount * ${type.width}); } - - - - - } } diff --git a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java index ea643f052f4..6a713a38434 100644 --- a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java +++ b/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java @@ -37,7 +37,6 @@ import parquet.schema.MessageType; import parquet.io.api.Binary; import io.netty.buffer.ByteBuf; -import org.apache.drill.exec.memory.TopLevelAllocator; import org.apache.drill.exec.record.BatchSchema; import org.apache.drill.exec.record.MaterializedField; import org.apache.drill.exec.vector.complex.fn.JsonOutput; @@ -61,6 +60,8 @@ * to output the data in string format instead of implementing addField for each type holder. * * This is useful for text format writers such as CSV, TSV etc. + * + * NB: Source code generated using FreeMarker template ${.template_name} */ public abstract class JSONOutputRecordWriter extends AbstractRecordWriter implements RecordWriter { diff --git a/exec/java-exec/src/main/codegen/templates/ListWriters.java b/exec/java-exec/src/main/codegen/templates/ListWriters.java index 6d26204c64c..f2683a3f024 100644 --- a/exec/java-exec/src/main/codegen/templates/ListWriters.java +++ b/exec/java-exec/src/main/codegen/templates/ListWriters.java @@ -36,9 +36,11 @@ <#include "/@includes/vv_imports.ftl" /> -/* This class is generated using freemarker and the ListWriters.java template */ +/* + * This class is generated using FreeMarker and the ${.template_name} template. + */ @SuppressWarnings("unused") -public class ${mode}ListWriter extends AbstractFieldWriter{ +public class ${mode}ListWriter extends AbstractFieldWriter { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${mode}ListWriter.class); static enum Mode { INIT, IN_MAP, IN_LIST <#list vv.types as type><#list type.minor as minor>, IN_${minor.class?upper_case} } @@ -48,7 +50,7 @@ static enum Mode { INIT, IN_MAP, IN_LIST <#list vv.types as type><#list type.min private Mode mode = Mode.INIT; private FieldWriter writer; protected RepeatedValueVector innerVector; - + <#if mode == "Repeated">private int currentChildIndex = 0; public ${mode}ListWriter(String name, ${containerClass} container, FieldWriter parent){ super(parent); @@ -62,36 +64,53 @@ static enum Mode { INIT, IN_MAP, IN_LIST <#list vv.types as type><#list type.min this.container = container; } - public void allocate(){ - if(writer != null){ + @Override + public void allocate() { + if(writer != null) { writer.allocate(); } - + <#if mode == "Repeated"> container.allocateNew(); } - - public void clear(){ - writer.clear(); + + @Override + public void clear() { + if (writer != null) { + writer.clear(); + } + } + + @Override + public void close() { + clear(); + container.close(); + if (innerVector != null) { + innerVector.close(); + } } + @Override public int getValueCapacity() { - return innerVector==null ? 0:innerVector.getValueCapacity(); + return innerVector == null ? 0 : innerVector.getValueCapacity(); } public void setValueCount(int count){ if(innerVector != null) innerVector.getMutator().setValueCount(count); } - - public MapWriter map(){ - switch(mode){ + + @Override + public MapWriter map() { + switch(mode) { case INIT: int vectorCount = container.size(); - RepeatedMapVector vector = container.addOrGet(name, RepeatedMapVector.TYPE, RepeatedMapVector.class); + final RepeatedMapVector vector = container.addOrGet(name, RepeatedMapVector.TYPE, RepeatedMapVector.class); innerVector = vector; writer = new RepeatedMapWriter(vector, this); - if(vectorCount != container.size()) writer.allocate(); + if(vectorCount != container.size()) { + writer.allocate(); + } writer.setPosition(${index}); mode = Mode.IN_MAP; return writer; @@ -102,15 +121,18 @@ public MapWriter map(){ throw UserException.unsupportedError().message(getUnsupportedErrorMsg("MAP", mode.name())).build(logger); } - - public ListWriter list(){ - switch(mode){ + + @Override + public ListWriter list() { + switch(mode) { case INIT: - int vectorCount = container.size(); - RepeatedListVector vector = container.addOrGet(name, RepeatedListVector.TYPE, RepeatedListVector.class); + final int vectorCount = container.size(); + final RepeatedListVector vector = container.addOrGet(name, RepeatedListVector.TYPE, RepeatedListVector.class); innerVector = vector; writer = new RepeatedListWriter(null, vector, this); - if(vectorCount != container.size()) writer.allocate(); + if(vectorCount != container.size()) { + writer.allocate(); + } writer.setPosition(${index}); mode = Mode.IN_LIST; return writer; @@ -121,23 +143,26 @@ public ListWriter list(){ throw UserException.unsupportedError().message(getUnsupportedErrorMsg("LIST", mode.name())).build(logger); } - + <#list vv.types as type><#list type.minor as minor> <#assign lowerName = minor.class?uncap_first /> <#assign upperName = minor.class?upper_case /> <#assign capName = minor.class?cap_first /> <#if lowerName == "int" ><#assign lowerName = "integer" /> - + private static final MajorType ${upperName}_TYPE = Types.repeated(MinorType.${upperName}); - - public ${capName}Writer ${lowerName}(){ - switch(mode){ + + @Override + public ${capName}Writer ${lowerName}() { + switch(mode) { case INIT: - int vectorCount = container.size(); - Repeated${capName}Vector vector = container.addOrGet(name, ${upperName}_TYPE, Repeated${capName}Vector.class); + final int vectorCount = container.size(); + final Repeated${capName}Vector vector = container.addOrGet(name, ${upperName}_TYPE, Repeated${capName}Vector.class); innerVector = vector; writer = new Repeated${capName}WriterImpl(vector, this); - if(vectorCount != container.size()) writer.allocate(); + if(vectorCount != container.size()) { + writer.allocate(); + } writer.setPosition(${index}); mode = Mode.IN_${upperName}; return writer; @@ -155,60 +180,55 @@ public MaterializedField getField() { } <#if mode == "Repeated"> - - public void start(){ - + + public void start() { final RepeatedListVector list = (RepeatedListVector) container; final RepeatedListVector.RepeatedMutator mutator = list.getMutator(); - + // make sure that the current vector can support the end position of this list. - if(container.getValueCapacity() <= idx()){ + if(container.getValueCapacity() <= idx()) { mutator.setValueCount(idx()+1); } - + // update the repeated vector to state that there is current+1 objects. - RepeatedListHolder h = new RepeatedListHolder(); + final RepeatedListHolder h = new RepeatedListHolder(); list.getAccessor().get(idx(), h); - if(h.start >= h.end){ - mutator.startNewValue(idx()); + if (h.start >= h.end) { + mutator.startNewValue(idx()); } currentChildIndex = container.getMutator().add(idx()); - if(writer != null){ - writer.setPosition(currentChildIndex); + if(writer != null) { + writer.setPosition(currentChildIndex); } } - - - - public void end(){ + + public void end() { // noop, we initialize state at start rather than end. } <#else> - - - public void setPosition(int index){ + + public void setPosition(int index) { super.setPosition(index); - if(writer != null) writer.setPosition(index); + if(writer != null) { + writer.setPosition(index); + } } - - public void start(){ + + public void start() { // noop } - - public void end(){ + + public void end() { // noop } - private String getUnsupportedErrorMsg(String expected, String found ){ - String f = found.substring(3); + private String getUnsupportedErrorMsg(String expected, String found) { + final String f = found.substring(3); return String.format("In a list of type %s, encountered a value of type %s. "+ "Drill does not support lists of different types.", f, expected ); } - - } +} - - diff --git a/exec/java-exec/src/main/codegen/templates/MapWriters.java b/exec/java-exec/src/main/codegen/templates/MapWriters.java index 06a6813d46d..7f57bd8ecbd 100644 --- a/exec/java-exec/src/main/codegen/templates/MapWriters.java +++ b/exec/java-exec/src/main/codegen/templates/MapWriters.java @@ -42,112 +42,135 @@ import com.google.common.collect.Maps; -/* This class is generated using freemarker and the MapWriters.java template */ +/* + * This class is generated using FreeMarker and the ${.template_name} template. + */ @SuppressWarnings("unused") -public class ${mode}MapWriter extends AbstractFieldWriter{ - +public class ${mode}MapWriter extends AbstractFieldWriter { + protected final ${containerClass} container; private final Map fields = Maps.newHashMap(); <#if mode == "Repeated">private int currentChildIndex = 0; - + public ${mode}MapWriter(${containerClass} container, FieldWriter parent) { super(parent); this.container = container; } + @Override public int getValueCapacity() { return container.getValueCapacity(); } + @Override public MaterializedField getField() { return container.getField(); } - public MapWriter map(String name){ - FieldWriter writer = fields.get(name.toLowerCase()); - if(writer == null){ + @Override + public MapWriter map(String name) { + FieldWriter writer = fields.get(name.toLowerCase()); + if(writer == null) { int vectorCount = container.size(); MapVector vector = container.addOrGet(name, MapVector.TYPE, MapVector.class); writer = new SingleMapWriter(vector, this); - if(vectorCount != container.size()) writer.allocate(); + if(vectorCount != container.size()) { + writer.allocate(); + } writer.setPosition(${index}); fields.put(name.toLowerCase(), writer); } return writer; } - - public void allocate(){ + + @Override + public void close() throws Exception { + clear(); +/* TODO(cwestin) experimental + for(final FieldWriter fieldWriter : fields.values()) { + fieldWriter.close(); + } +*/ + container.close(); + } + + @Override + public void allocate() { container.allocateNew(); - for(FieldWriter w : fields.values()){ + for(final FieldWriter w : fields.values()) { w.allocate(); } } - - public void clear(){ + + @Override + public void clear() { container.clear(); - for(FieldWriter w : fields.values()){ + for(final FieldWriter w : fields.values()) { w.clear(); } } - - public ListWriter list(String name){ + + @Override + public ListWriter list(String name) { FieldWriter writer = fields.get(name.toLowerCase()); - if(writer == null){ + if(writer == null) { writer = new SingleListWriter(name, container, this); writer.setPosition(${index}); fields.put(name.toLowerCase(), writer); } return writer; } - <#if mode == "Repeated"> - public void start(){ + public void start() { // update the repeated vector to state that there is current+1 objects. - final RepeatedMapHolder h = new RepeatedMapHolder(); final RepeatedMapVector map = (RepeatedMapVector) container; final RepeatedMapVector.Mutator mutator = map.getMutator(); - - // make sure that the current vector can support the end position of this list. - if(container.getValueCapacity() <= idx()){ + + // Make sure that the current vector can support the end position of this list. + if(container.getValueCapacity() <= idx()) { mutator.setValueCount(idx()+1); } map.getAccessor().get(idx(), h); - if(h.start >= h.end){ - container.getMutator().startNewValue(idx()); + if (h.start >= h.end) { + container.getMutator().startNewValue(idx()); } currentChildIndex = container.getMutator().add(idx()); - for(FieldWriter w: fields.values()){ - w.setPosition(currentChildIndex); + for(final FieldWriter w : fields.values()) { + w.setPosition(currentChildIndex); } } - - public void end(){ + + public void end() { // noop } <#else> - public void setValueCount(int count){ + public void setValueCount(int count) { container.getMutator().setValueCount(count); } - public void setPosition(int index){ + @Override + public void setPosition(int index) { super.setPosition(index); - for(FieldWriter w: fields.values()){ - w.setPosition(index); + for(final FieldWriter w: fields.values()) { + w.setPosition(index); } } - public void start(){ + + @Override + public void start() { } - - public void end(){ - // noop + + @Override + public void end() { } + - + <#list vv.types as type><#list type.minor as minor> <#assign lowerName = minor.class?uncap_first /> <#if lowerName == "int" ><#assign lowerName = "integer" /> @@ -157,22 +180,23 @@ public void end(){ <#assign vectName = "Nullable${capName}" /> <#if minor.class?starts_with("Decimal") > - public ${minor.class}Writer ${lowerName}(String name){ + public ${minor.class}Writer ${lowerName}(String name) { // returns existing writer - FieldWriter writer = fields.get(name.toLowerCase()); + final FieldWriter writer = fields.get(name.toLowerCase()); assert writer != null; return writer; } - public ${minor.class}Writer ${lowerName}(String name, int scale, int precision){ + public ${minor.class}Writer ${lowerName}(String name, int scale, int precision) { final MajorType ${upperName}_TYPE = Types.withScaleAndPrecision(MinorType.${upperName}, DataMode.OPTIONAL, scale, precision); <#else> private static final MajorType ${upperName}_TYPE = Types.optional(MinorType.${upperName}); - public ${minor.class}Writer ${lowerName}(String name){ + @Override + public ${minor.class}Writer ${lowerName}(String name) { FieldWriter writer = fields.get(name.toLowerCase()); - if(writer == null){ - ${vectName}Vector vector = container.addOrGet(name, ${upperName}_TYPE, ${vectName}Vector.class); + if(writer == null) { + final ${vectName}Vector vector = container.addOrGet(name, ${upperName}_TYPE, ${vectName}Vector.class); vector.allocateNewSafe(); writer = new ${vectName}WriterImpl(vector, this); writer.setPosition(${index}); @@ -180,13 +204,8 @@ public void end(){ } return writer; } - - - - + } - - diff --git a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java index 7fa0d551fec..4bdde163f3e 100644 --- a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java +++ b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java @@ -42,10 +42,10 @@ * are first checked against a fixed length vector of boolean values. Then the element is retrieved * from the base class (if not null). * - * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker. + * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker. */ @SuppressWarnings("unused") -public final class ${className} extends BaseDataValueVector implements <#if type.major == "VarLen">VariableWidth<#else>FixedWidthVector, NullableVector{ +public final class ${className} extends BaseDataValueVector implements <#if type.major == "VarLen">VariableWidth<#else>FixedWidthVector, NullableVector { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${className}.class); private final FieldReader reader = new Nullable${minor.class}ReaderImpl(Nullable${minor.class}Vector.this); @@ -60,26 +60,34 @@ public final class ${className} extends BaseDataValueVector implements <#if type } @Override - public FieldReader getReader(){ + public FieldReader getReader() { return reader; } - public int getValueCapacity(){ + @Override + public int getValueCapacity() { return Math.min(bits.getValueCapacity(), values.getValueCapacity()); } @Override public DrillBuf[] getBuffers(boolean clear) { - DrillBuf[] buffers = ObjectArrays.concat(bits.getBuffers(false), values.getBuffers(false), DrillBuf.class); + final DrillBuf[] buffers = ObjectArrays.concat(bits.getBuffers(false), values.getBuffers(false), DrillBuf.class); if (clear) { - for (DrillBuf buffer:buffers) { - buffer.retain(); + for (final DrillBuf buffer:buffers) { + buffer.retain(1); } clear(); } return buffers; } + @Override + public void close() { + bits.close(); + values.close(); + super.close(); + } + @Override public void clear() { bits.clear(); @@ -87,7 +95,8 @@ public void clear() { super.clear(); } - public int getBufferSize(){ + @Override + public int getBufferSize() { return values.getBufferSize() + bits.getBufferSize(); } @@ -96,6 +105,7 @@ public DrillBuf getBuffer() { return values.getBuffer(); } + @Override public ${valuesName} getValuesVector() { return values; } @@ -116,8 +126,9 @@ public SerializedField getMetadata() { .build(); } + @Override public void allocateNew() { - if(!allocateNewSafe()){ + if(!allocateNewSafe()) { throw new OutOfMemoryRuntimeException("Failure while allocating buffer."); } } @@ -158,7 +169,7 @@ public void allocateNew(int totalBytes, int valueCount) { } @Override - public int load(int dataBytes, int valueCount, DrillBuf buf){ + public int load(int dataBytes, int valueCount, DrillBuf buf) { clear(); int loaded = bits.load(valueCount, buf); @@ -166,24 +177,24 @@ public int load(int dataBytes, int valueCount, DrillBuf buf){ buf = buf.slice(loaded, buf.capacity() - loaded); dataBytes -= loaded; loaded += values.load(dataBytes, valueCount, buf); - this.mutator.lastSet = valueCount; + mutator.lastSet = valueCount; return loaded; } @Override public void load(SerializedField metadata, DrillBuf buffer) { - assert this.field.matches(metadata) : String.format("The field %s doesn't match the provided metadata %s.", this.field, metadata); - int loaded = load(metadata.getBufferLength(), metadata.getValueCount(), buffer); + assert field.matches(metadata) : String.format("The field %s doesn't match the provided metadata %s.", field, metadata); + final int loaded = load(metadata.getBufferLength(), metadata.getValueCount(), buffer); assert metadata.getBufferLength() == loaded : String.format("Expected to load %d bytes but actually loaded %d bytes", metadata.getBufferLength(), loaded); } @Override - public int getByteCapacity(){ + public int getByteCapacity() { return values.getByteCapacity(); } @Override - public int getCurrentSizeInBytes(){ + public int getCurrentSizeInBytes() { return values.getCurrentSizeInBytes(); } @@ -242,13 +253,14 @@ public void allocateNew(int valueCount) { /** * {@inheritDoc} */ + @Override public void zeroVector() { - this.values.zeroVector(); - this.bits.zeroVector(); + values.zeroVector(); + bits.zeroVector(); } @Override - public int load(int valueCount, DrillBuf buf){ + public int load(int valueCount, DrillBuf buf) { clear(); int loaded = bits.load(valueCount, buf); @@ -260,26 +272,29 @@ public int load(int valueCount, DrillBuf buf){ @Override public void load(SerializedField metadata, DrillBuf buffer) { - assert this.field.matches(metadata); - int loaded = load(metadata.getValueCount(), buffer); + assert field.matches(metadata); + final int loaded = load(metadata.getValueCount(), buffer); assert metadata.getBufferLength() == loaded; } - public TransferPair getTransferPair(){ + @Override + public TransferPair getTransferPair() { return new TransferImpl(getField()); } + + @Override public TransferPair getTransferPair(FieldReference ref){ return new TransferImpl(getField().withPath(ref)); } + @Override public TransferPair makeTransferPair(ValueVector to) { return new TransferImpl((Nullable${minor.class}Vector) to); } - - public void transferTo(Nullable${minor.class}Vector target){ + public void transferTo(Nullable${minor.class}Vector target) { bits.transferTo(target.bits); values.transferTo(target.values); <#if type.major == "VarLen"> @@ -296,25 +311,28 @@ public void splitAndTransferTo(int startIndex, int length, Nullable${minor.class } - private class TransferImpl implements TransferPair{ + private class TransferImpl implements TransferPair { Nullable${minor.class}Vector to; - public TransferImpl(MaterializedField field){ - this.to = new Nullable${minor.class}Vector(field, allocator); + public TransferImpl(MaterializedField field) { + to = new Nullable${minor.class}Vector(field, allocator); } - public TransferImpl(Nullable${minor.class}Vector to){ + public TransferImpl(Nullable${minor.class}Vector to) { this.to = to; } - public Nullable${minor.class}Vector getTo(){ + @Override + public Nullable${minor.class}Vector getTo() { return to; } - public void transfer(){ + @Override + public void transfer() { transferTo(to); } + @Override public void splitAndTransfer(int startIndex, int length) { splitAndTransferTo(startIndex, length, to); } @@ -325,31 +343,35 @@ public void copyValueSafe(int fromIndex, int toIndex) { } } - public Accessor getAccessor(){ + @Override + public Accessor getAccessor() { return accessor; } - public Mutator getMutator(){ + @Override + public Mutator getMutator() { return mutator; } - public ${minor.class}Vector convertToRequiredVector(){ + public ${minor.class}Vector convertToRequiredVector() { ${minor.class}Vector v = new ${minor.class}Vector(getField().getOtherNullableVersion(), allocator); + if (v.data != null) { + v.data.release(1); + } v.data = values.data; - v.data.retain(); + v.data.retain(1); clear(); return v; } - - public void copyFrom(int fromIndex, int thisIndex, Nullable${minor.class}Vector from){ - if (!from.getAccessor().isNull(fromIndex)) { - mutator.set(thisIndex, from.getAccessor().get(fromIndex)); + public void copyFrom(int fromIndex, int thisIndex, Nullable${minor.class}Vector from) { + final Accessor fromAccessor = from.getAccessor(); + if (!fromAccessor.isNull(fromIndex)) { + mutator.set(thisIndex, fromAccessor.get(fromIndex)); } } - - public void copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from){ + public void copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from) { <#if type.major == "VarLen"> mutator.fillEmpties(thisIndex); @@ -357,7 +379,7 @@ public void copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from bits.getMutator().setSafe(thisIndex, 1); } - public void copyFromSafe(int fromIndex, int thisIndex, Nullable${minor.class}Vector from){ + public void copyFromSafe(int fromIndex, int thisIndex, Nullable${minor.class}Vector from) { <#if type.major == "VarLen"> mutator.fillEmpties(thisIndex); @@ -366,7 +388,6 @@ public void copyFromSafe(int fromIndex, int thisIndex, Nullable${minor.class}Vec } public final class Accessor extends BaseDataValueVector.BaseAccessor <#if type.major = "VarLen">implements VariableWidthVector.VariableWidthAccessor { - final UInt1Vector.Accessor bAccessor = bits.getAccessor(); final ${valuesName}.Accessor vAccessor = values.getAccessor(); @@ -379,30 +400,32 @@ public final class Accessor extends BaseDataValueVector.BaseAccessor <#if type.m */ public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType} get(int index) { if (isNull(index)) { - throw new IllegalStateException("Can't get a null value"); + throw new IllegalStateException("Can't get a null value"); } return vAccessor.get(index); } + @Override public boolean isNull(int index) { return isSet(index) == 0; } - public int isSet(int index){ + public int isSet(int index) { return bAccessor.get(index); } <#if type.major == "VarLen"> - public long getStartEnd(int index){ + public long getStartEnd(int index) { return vAccessor.getStartEnd(index); } + @Override public int getValueLength(int index) { return values.getAccessor().getValueLength(index); } - public void get(int index, Nullable${minor.class}Holder holder){ + public void get(int index, Nullable${minor.class}Holder holder) { vAccessor.get(index, holder); holder.isSet = bAccessor.get(index); @@ -416,7 +439,7 @@ public void get(int index, Nullable${minor.class}Holder holder){ public ${friendlyType} getObject(int index) { if (isNull(index)) { return null; - }else{ + } else { return vAccessor.getObject(index); } } @@ -425,32 +448,33 @@ public void get(int index, Nullable${minor.class}Holder holder){ public StringBuilder getAsStringBuilder(int index) { if (isNull(index)) { return null; - }else{ + } else { return vAccessor.getAsStringBuilder(index); } } - public int getValueCount(){ + @Override + public int getValueCount() { return bits.getAccessor().getValueCount(); } - public void reset(){} + public void reset() {} } public final class Mutator extends BaseDataValueVector.BaseMutator implements NullableVectorDefinitionSetter<#if type.major = "VarLen">, VariableWidthVector.VariableWidthMutator { - private int setCount; <#if type.major = "VarLen"> private int lastSet = -1; - private Mutator(){ + private Mutator() { } - public ${valuesName} getVectorWithValues(){ + public ${valuesName} getVectorWithValues() { return values; } - public void setIndexDefined(int index){ + @Override + public void setIndexDefined(int index) { bits.getMutator().set(index, 1); } @@ -462,27 +486,32 @@ public void setIndexDefined(int index){ */ public void set(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType} value) { setCount++; + final ${valuesName}.Mutator valuesMutator = values.getMutator(); + final UInt1Vector.Mutator bitsMutator = bits.getMutator(); <#if type.major == "VarLen"> for (int i = lastSet + 1; i < index; i++) { - values.getMutator().set(i, new byte[]{}); + valuesMutator.set(i, emptyByteArray); } - bits.getMutator().set(index, 1); - values.getMutator().set(index, value); + bitsMutator.set(index, 1); + valuesMutator.set(index, value); <#if type.major == "VarLen">lastSet = index; } <#if type.major == "VarLen"> - private void fillEmpties(int index){ - for (int i = lastSet; i < index; i++) { - values.getMutator().setSafe(i+1, new byte[]{}); + + private void fillEmpties(int index) { + final ${valuesName}.Mutator valuesMutator = values.getMutator(); + for(int i = lastSet; i < index; i++) { + valuesMutator.setSafe(i + 1, emptyByteArray); } - if (index > bits.getValueCapacity()) { + while(index > bits.getValueCapacity()) { bits.reAlloc(); } lastSet = index; } + @Override public void setValueLengthSafe(int index, int length) { values.getMutator().setValueLengthSafe(index, length); } @@ -514,38 +543,40 @@ public void setSafe(int index, ByteBuffer value, int start, int length) { } - public void setNull(int index){ + public void setNull(int index) { bits.getMutator().setSafe(index, 0); } - public void setSkipNull(int index, ${minor.class}Holder holder){ + public void setSkipNull(int index, ${minor.class}Holder holder) { values.getMutator().set(index, holder); } - public void setSkipNull(int index, Nullable${minor.class}Holder holder){ + public void setSkipNull(int index, Nullable${minor.class}Holder holder) { values.getMutator().set(index, holder); } - public void set(int index, Nullable${minor.class}Holder holder){ + public void set(int index, Nullable${minor.class}Holder holder) { + final ${valuesName}.Mutator valuesMutator = values.getMutator(); <#if type.major == "VarLen"> for (int i = lastSet + 1; i < index; i++) { - values.getMutator().set(i, new byte[]{}); + valuesMutator.set(i, emptyByteArray); } bits.getMutator().set(index, holder.isSet); - values.getMutator().set(index, holder); + valuesMutator.set(index, holder); <#if type.major == "VarLen">lastSet = index; } - public void set(int index, ${minor.class}Holder holder){ + public void set(int index, ${minor.class}Holder holder) { + final ${valuesName}.Mutator valuesMutator = values.getMutator(); <#if type.major == "VarLen"> for (int i = lastSet + 1; i < index; i++) { - values.getMutator().set(i, new byte[]{}); + valuesMutator.set(i, emptyByteArray); } bits.getMutator().set(index, 1); - values.getMutator().set(index, holder); + valuesMutator.set(index, holder); <#if type.major == "VarLen">lastSet = index; } @@ -554,14 +585,15 @@ public boolean isSafe(int outIndex) { } <#assign fields = minor.fields!type.fields /> - public void set(int index, int isSet<#list fields as field><#if field.include!true >, ${field.type} ${field.name}Field ){ + public void set(int index, int isSet<#list fields as field><#if field.include!true >, ${field.type} ${field.name}Field ) { + final ${valuesName}.Mutator valuesMutator = values.getMutator(); <#if type.major == "VarLen"> for (int i = lastSet + 1; i < index; i++) { - values.getMutator().set(i, new byte[]{}); + valuesMutator.set(i, emptyByteArray); } bits.getMutator().set(index, isSet); - values.getMutator().set(index<#list fields as field><#if field.include!true >, ${field.name}Field); + valuesMutator.set(index<#list fields as field><#if field.include!true >, ${field.name}Field); <#if type.major == "VarLen">lastSet = index; } @@ -611,6 +643,7 @@ public void setSafe(int index, ${minor.javaType!type.javaType} value) { + @Override public void setValueCount(int valueCount) { assert valueCount >= 0; <#if type.major == "VarLen"> @@ -620,18 +653,19 @@ public void setValueCount(int valueCount) { bits.getMutator().setValueCount(valueCount); } - public void generateTestData(int valueCount){ + @Override + public void generateTestData(int valueCount) { bits.getMutator().generateTestDataAlt(valueCount); values.getMutator().generateTestData(valueCount); <#if type.major = "VarLen">lastSet = valueCount; setValueCount(valueCount); } - public void reset(){ + @Override + public void reset() { setCount = 0; <#if type.major = "VarLen">lastSet = -1; } - } } diff --git a/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java index 35777b0ebec..787edc2dee8 100644 --- a/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java +++ b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java @@ -43,7 +43,6 @@ import parquet.schema.MessageType; import parquet.io.api.Binary; import io.netty.buffer.DrillBuf; -import org.apache.drill.exec.memory.TopLevelAllocator; import org.apache.drill.exec.record.BatchSchema; import org.apache.drill.exec.record.MaterializedField; @@ -65,6 +64,8 @@ * to output the data in string format instead of implementing addField for each type holder. * * This is useful for text format writers such as CSV, TSV etc. + * + * NB: Source code generated using FreeMarker template ${.template_name} */ public abstract class ParquetOutputRecordWriter extends AbstractRecordWriter implements RecordWriter { diff --git a/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java b/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java index 6431d8808f4..2f5ec18b028 100644 --- a/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java +++ b/exec/java-exec/src/main/codegen/templates/RepeatedValueVectors.java @@ -35,27 +35,24 @@ <#include "/@includes/vv_imports.ftl" /> - -@SuppressWarnings("unused") /** * Repeated${minor.class} implements a vector with multple values per row (e.g. JSON array or * repeated protobuf field). The implementation uses two additional value vectors; one to convert * the index offset to the underlying element offset, and another to store the number of values * in the vector. * - * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker. + * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker. */ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector implements Repeated<#if type.major == "VarLen">VariableWidth<#else>FixedWidthVectorLike { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Repeated${minor.class}Vector.class); + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Repeated${minor.class}Vector.class); // we maintain local reference to concrete vector type for performance reasons. private ${minor.class}Vector values; private final FieldReader reader = new Repeated${minor.class}ReaderImpl(Repeated${minor.class}Vector.this); private final Mutator mutator = new Mutator(); private final Accessor accessor = new Accessor(); - - + public Repeated${minor.class}Vector(MaterializedField field, BufferAllocator allocator) { super(field, allocator); addOrGetVector(VectorDescriptor.create(Types.required(field.getType().getMinorType()))); @@ -72,17 +69,17 @@ public Accessor getAccessor() { } @Override - public FieldReader getReader(){ + public FieldReader getReader() { return reader; } @Override - public ${minor.class}Vector getDataVector(){ + public ${minor.class}Vector getDataVector() { return values; } @Override - public TransferPair getTransferPair(){ + public TransferPair getTransferPair() { return new TransferImpl(getField()); } @@ -105,7 +102,7 @@ public TransferPair makeTransferPair(ValueVector to) { return result; } - public void transferTo(Repeated${minor.class}Vector target){ + public void transferTo(Repeated${minor.class}Vector target) { target.clear(); offsets.transferTo(target.offsets); values.transferTo(target.values); @@ -115,11 +112,11 @@ public void transferTo(Repeated${minor.class}Vector target){ public void splitAndTransferTo(final int startIndex, final int groups, Repeated${minor.class}Vector to) { final UInt4Vector.Accessor a = offsets.getAccessor(); final UInt4Vector.Mutator m = to.offsets.getMutator(); - - final int startPos = offsets.getAccessor().get(startIndex); - final int endPos = offsets.getAccessor().get(startIndex + groups); + + final int startPos = a.get(startIndex); + final int endPos = a.get(startIndex + groups); final int valuesToCopy = endPos - startPos; - + values.splitAndTransferTo(startPos, valuesToCopy, to.values); to.offsets.clear(); to.offsets.allocateNew(groups + 1); @@ -130,53 +127,57 @@ public void splitAndTransferTo(final int startIndex, final int groups, Repeated$ } m.setValueCount(groups == 0 ? 0 : groups + 1); } - - private class TransferImpl implements TransferPair{ - Repeated${minor.class}Vector to; - - public TransferImpl(MaterializedField field){ + + private class TransferImpl implements TransferPair { + final Repeated${minor.class}Vector to; + + public TransferImpl(MaterializedField field) { this.to = new Repeated${minor.class}Vector(field, allocator); } - public TransferImpl(Repeated${minor.class}Vector to){ + public TransferImpl(Repeated${minor.class}Vector to) { this.to = to; } - public Repeated${minor.class}Vector getTo(){ + @Override + public Repeated${minor.class}Vector getTo() { return to; } - - public void transfer(){ + + @Override + public void transfer() { transferTo(to); } + @Override public void splitAndTransfer(int startIndex, int length) { splitAndTransferTo(startIndex, length, to); } - + @Override public void copyValueSafe(int fromIndex, int toIndex) { to.copyFromSafe(fromIndex, toIndex, Repeated${minor.class}Vector.this); } } - public void copyFrom(int inIndex, int outIndex, Repeated${minor.class}Vector v){ - final int count = v.getAccessor().getInnerValueCountAt(inIndex); - getMutator().startNewValue(outIndex); + public void copyFrom(int inIndex, int outIndex, Repeated${minor.class}Vector v) { + final Accessor vAccessor = v.getAccessor(); + final int count = vAccessor.getInnerValueCountAt(inIndex); + mutator.startNewValue(outIndex); for (int i = 0; i < count; i++) { - getMutator().add(outIndex, v.getAccessor().get(inIndex, i)); + mutator.add(outIndex, vAccessor.get(inIndex, i)); } } - public void copyFromSafe(int inIndex, int outIndex, Repeated${minor.class}Vector v){ - final int count = v.getAccessor().getInnerValueCountAt(inIndex); - getMutator().startNewValue(outIndex); + public void copyFromSafe(int inIndex, int outIndex, Repeated${minor.class}Vector v) { + final Accessor vAccessor = v.getAccessor(); + final int count = vAccessor.getInnerValueCountAt(inIndex); + mutator.startNewValue(outIndex); for (int i = 0; i < count; i++) { - getMutator().addSafe(outIndex, v.getAccessor().get(inIndex, i)); + mutator.addSafe(outIndex, vAccessor.get(inIndex, i)); } } - public boolean allocateNewSafe() { /* boolean to keep track if all the memory allocation were successful * Used in the case of composite vectors when we need to allocate multiple @@ -197,7 +198,8 @@ public boolean allocateNewSafe() { mutator.reset(); return true; } - + + @Override public void allocateNew() { try { offsets.allocateNew(); @@ -216,10 +218,10 @@ protected SerializedField.Builder getMetadataBuilder() { return super.getMetadataBuilder() .setVarByteLength(values.getVarByteLength()); } - + public void allocateNew(int totalBytes, int valueCount, int innerValueCount) { try { - offsets.allocateNew(valueCount+1); + offsets.allocateNew(valueCount + 1); values.allocateNew(totalBytes, innerValueCount); } catch (OutOfMemoryRuntimeException e) { clear(); @@ -235,6 +237,7 @@ public int getByteCapacity(){ <#else> + @Override public void allocateNew(int valueCount, int innerValueCount) { clear(); /* boolean to keep track if all the memory allocation were successful @@ -244,7 +247,7 @@ public void allocateNew(int valueCount, int innerValueCount) { */ boolean success = false; try { - offsets.allocateNew(valueCount+1); + offsets.allocateNew(valueCount + 1); values.allocateNew(innerValueCount); } catch(OutOfMemoryRuntimeException e){ clear(); @@ -253,27 +256,29 @@ public void allocateNew(int valueCount, int innerValueCount) { offsets.zeroVector(); mutator.reset(); } - + // This is declared a subclass of the accessor declared inside of FixedWidthVector, this is also used for // 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 extends BaseRepeatedValueVector.BaseRepeatedAccessor { - + @Override public List<${friendlyType}> getObject(int index) { - List<${friendlyType}> vals = new JsonStringArrayList(); - int start = offsets.getAccessor().get(index); - int end = offsets.getAccessor().get(index+1); - for(int i = start; i < end; i++){ - vals.add(values.getAccessor().getObject(i)); + final List<${friendlyType}> vals = new JsonStringArrayList<>(); + final UInt4Vector.Accessor offsetsAccessor = offsets.getAccessor(); + final int start = offsetsAccessor.get(index); + final int end = offsetsAccessor.get(index + 1); + final ${minor.class}Vector.Accessor valuesAccessor = values.getAccessor(); + for(int i = start; i < end; i++) { + vals.add(valuesAccessor.getObject(i)); } return vals; } - - public ${friendlyType} getSingleObject(int index, int arrayIndex){ - int start = offsets.getAccessor().get(index); + + public ${friendlyType} getSingleObject(int index, int arrayIndex) { + final int start = offsets.getAccessor().get(index); return values.getAccessor().getObject(start + arrayIndex); } @@ -291,21 +296,21 @@ public final class Accessor extends BaseRepeatedValueVector.BaseRepeatedAccessor return values.getAccessor().get(offsets.getAccessor().get(index) + positionIndex); } - public void get(int index, Repeated${minor.class}Holder holder){ + public void get(int index, Repeated${minor.class}Holder holder) { holder.start = offsets.getAccessor().get(index); holder.end = offsets.getAccessor().get(index+1); holder.vector = values; } public void get(int index, int positionIndex, ${minor.class}Holder holder) { - int offset = offsets.getAccessor().get(index); + final int offset = offsets.getAccessor().get(index); assert offset >= 0; assert positionIndex < getInnerValueCountAt(index); values.getAccessor().get(offset + positionIndex, holder); } - + public void get(int index, int positionIndex, Nullable${minor.class}Holder holder) { - int offset = offsets.getAccessor().get(index); + final int offset = offsets.getAccessor().get(index); assert offset >= 0; if (positionIndex >= getInnerValueCountAt(index)) { holder.isSet = 0; @@ -314,10 +319,9 @@ public void get(int index, int positionIndex, Nullable${minor.class}Holder holde values.getAccessor().get(offset + positionIndex, holder); } } - - public final class Mutator extends BaseRepeatedValueVector.BaseRepeatedMutator implements RepeatedMutator { - private Mutator() { } + public final class Mutator extends BaseRepeatedValueVector.BaseRepeatedMutator implements RepeatedMutator { + private Mutator() {} /** * Add an element to the given record index. This is similar to the set() method in other @@ -338,7 +342,7 @@ public void addSafe(int index, byte[] bytes) { } public void addSafe(int index, byte[] bytes, int start, int length) { - int nextOffset = offsets.getAccessor().get(index+1); + final int nextOffset = offsets.getAccessor().get(index+1); values.getMutator().setSafe(nextOffset, bytes, start, length); offsets.getMutator().setSafe(index+1, nextOffset+1); } @@ -346,76 +350,77 @@ public void addSafe(int index, byte[] bytes, int start, int length) { <#else> public void addSafe(int index, ${minor.javaType!type.javaType} srcValue) { - int nextOffset = offsets.getAccessor().get(index+1); + final int nextOffset = offsets.getAccessor().get(index+1); values.getMutator().setSafe(nextOffset, srcValue); offsets.getMutator().setSafe(index+1, nextOffset+1); } - + - - public void setSafe(int index, Repeated${minor.class}Holder h){ - ${minor.class}Holder ih = new ${minor.class}Holder(); - getMutator().startNewValue(index); + public void setSafe(int index, Repeated${minor.class}Holder h) { + final ${minor.class}Holder ih = new ${minor.class}Holder(); + final ${minor.class}Vector.Accessor hVectorAccessor = h.vector.getAccessor(); + mutator.startNewValue(index); for(int i = h.start; i < h.end; i++){ - h.vector.getAccessor().get(i, ih); - getMutator().addSafe(index, ih); + hVectorAccessor.get(i, ih); + mutator.addSafe(index, ih); } } - - public void addSafe(int index, ${minor.class}Holder holder){ + + public void addSafe(int index, ${minor.class}Holder holder) { int nextOffset = offsets.getAccessor().get(index+1); values.getMutator().setSafe(nextOffset, holder); offsets.getMutator().setSafe(index+1, nextOffset+1); } - - public void addSafe(int index, Nullable${minor.class}Holder holder){ - int nextOffset = offsets.getAccessor().get(index+1); + + public void addSafe(int index, Nullable${minor.class}Holder holder) { + final int nextOffset = offsets.getAccessor().get(index+1); values.getMutator().setSafe(nextOffset, holder); offsets.getMutator().setSafe(index+1, nextOffset+1); } - + <#if (fields?size > 1) && !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")> - public void addSafe(int arrayIndex, <#list fields as field>${field.type} ${field.name}<#if field_has_next>, ){ + public void addSafe(int arrayIndex, <#list fields as field>${field.type} ${field.name}<#if field_has_next>, ) { int nextOffset = offsets.getAccessor().get(arrayIndex+1); values.getMutator().setSafe(nextOffset, <#list fields as field>${field.name}<#if field_has_next>, ); offsets.getMutator().setSafe(arrayIndex+1, nextOffset+1); } - - protected void add(int index, ${minor.class}Holder holder){ + + protected void add(int index, ${minor.class}Holder holder) { int nextOffset = offsets.getAccessor().get(index+1); values.getMutator().set(nextOffset, holder); offsets.getMutator().set(index+1, nextOffset+1); } - - public void add(int index, Repeated${minor.class}Holder holder){ - + + public void add(int index, Repeated${minor.class}Holder holder) { + ${minor.class}Vector.Accessor accessor = holder.vector.getAccessor(); ${minor.class}Holder innerHolder = new ${minor.class}Holder(); - - for(int i = holder.start; i < holder.end; i++){ + + for(int i = holder.start; i < holder.end; i++) { accessor.get(i, innerHolder); add(index, innerHolder); } } - public void generateTestData(final int valCount){ - int[] sizes = {1,2,0,6}; + @Override + public void generateTestData(final int valCount) { + final int[] sizes = {1, 2, 0, 6}; int size = 0; int runningOffset = 0; - for(int i =1; i < valCount+1; i++, size++){ + final UInt4Vector.Mutator offsetsMutator = offsets.getMutator(); + for(int i = 1; i < valCount + 1; i++, size++) { runningOffset += sizes[size % sizes.length]; - offsets.getMutator().set(i, runningOffset); + offsetsMutator.set(i, runningOffset); } - values.getMutator().generateTestData(valCount*9); + values.getMutator().generateTestData(valCount * 9); setValueCount(size); } - public void reset(){ - + @Override + public void reset() { } - } } diff --git a/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java index c175900cc61..6d92a2fb05f 100644 --- a/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java +++ b/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java @@ -30,7 +30,6 @@ import com.google.common.collect.Lists; import org.apache.drill.exec.expr.TypeHelper; import org.apache.drill.exec.expr.holders.*; -import org.apache.drill.exec.memory.TopLevelAllocator; import org.apache.drill.exec.memory.BufferAllocator; import org.apache.drill.exec.record.BatchSchema; import org.apache.drill.exec.record.VectorAccessible; @@ -50,6 +49,8 @@ * to output the data in string format instead of implementing addField for each type holder. * * This is useful for text format writers such as CSV, TSV etc. + * + * NB: Source code generated using FreeMarker template ${.template_name} */ public abstract class StringOutputRecordWriter extends AbstractRecordWriter { diff --git a/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java b/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java index fc08ac664ca..ac4d5390a0d 100644 --- a/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java +++ b/exec/java-exec/src/main/codegen/templates/VariableLengthVectors.java @@ -29,7 +29,6 @@ <#assign friendlyType = (minor.friendlyType!minor.boxedType!type.boxedType) /> - <#if type.major == "VarLen"> <@pp.changeOutputFile name="/org/apache/drill/exec/vector/${minor.class}Vector.java" /> @@ -47,11 +46,10 @@ * The width of each element is ${type.width} byte(s) * The equivalent Java primitive is '${minor.javaType!type.javaType}' * - * Source code generated using FreeMarker template ${.template_name} + * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker. */ -@SuppressWarnings("unused") -public final class ${minor.class}Vector extends BaseDataValueVector implements VariableWidthVector{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class); +public final class ${minor.class}Vector extends BaseDataValueVector implements VariableWidthVector { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class); private static final int DEFAULT_RECORD_BYTE_COUNT = 8; private static final int INITIAL_BYTE_COUNT = 4096 * DEFAULT_RECORD_BYTE_COUNT; @@ -68,7 +66,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V private final UInt${type.width}Vector.Accessor oAccessor; - private int allocationSizeInBytes = INITIAL_BYTE_COUNT; private int allocationMonitor = 0; @@ -81,27 +78,29 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V } @Override - public FieldReader getReader(){ + public FieldReader getReader() { return reader; } - public int getBufferSize(){ - if (getAccessor().getValueCount() == 0) return 0; + @Override + public int getBufferSize() { + if (getAccessor().getValueCount() == 0) { + return 0; + } return offsetVector.getBufferSize() + data.writerIndex(); } - int getSizeFromCount(int valueCount) { - return valueCount * ${type.width}; - } - - public int getValueCapacity(){ + @Override + public int getValueCapacity() { return offsetVector.getValueCapacity() - 1; } - public int getByteCapacity(){ + @Override + public int getByteCapacity() { return data.capacity(); } + @Override public int getCurrentSizeInBytes() { return offsetVector.getAccessor().get(getAccessor().getValueCount()); } @@ -110,31 +109,34 @@ public int getCurrentSizeInBytes() { * Return the number of bytes contained in the current var len byte vector. * @return */ - public int getVarByteLength(){ + public int getVarByteLength() { final int valueCount = getAccessor().getValueCount(); - if(valueCount == 0) return 0; + if(valueCount == 0) { + return 0; + } return offsetVector.getAccessor().get(valueCount); } @Override public SerializedField getMetadata() { - return getMetadataBuilder() // - .setValueCount(getAccessor().getValueCount()) // - .setVarByteLength(getVarByteLength()) // - .setBufferLength(getBufferSize()) // + return getMetadataBuilder() + .setValueCount(getAccessor().getValueCount()) + .setVarByteLength(getVarByteLength()) + .setBufferLength(getBufferSize()) .build(); } - public int load(int dataBytes, int valueCount, DrillBuf buf){ - if(valueCount == 0){ - allocateNew(0,0); + @Override + public int load(int dataBytes, int valueCount, DrillBuf buf) { + if(valueCount == 0) { + allocateNew(0, 0); return 0; } clear(); - int loaded = offsetVector.load(valueCount+1, buf); + final int loaded = offsetVector.load(valueCount + 1, buf); data = buf.slice(loaded, dataBytes - loaded); - data.retain(); - return dataBytes; + data.retain(1); + return dataBytes; } @Override @@ -150,109 +152,115 @@ public void clear() { offsetVector.clear(); } - @Override public DrillBuf[] getBuffers(boolean clear) { - DrillBuf[] buffers = ObjectArrays.concat(offsetVector.getBuffers(false), super.getBuffers(false), DrillBuf.class); + final 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(); + for (final DrillBuf buffer:buffers) { + buffer.retain(1); } clear(); } return buffers; } - public long getOffsetAddr(){ + public long getOffsetAddr() { return offsetVector.getBuffer().memoryAddress(); } - public UInt${type.width}Vector getOffsetVector(){ + public UInt${type.width}Vector getOffsetVector() { return offsetVector; } - public TransferPair getTransferPair(){ + @Override + public TransferPair getTransferPair() { return new TransferImpl(getField()); } + + @Override public TransferPair getTransferPair(FieldReference ref){ return new TransferImpl(getField().withPath(ref)); } + @Override public TransferPair makeTransferPair(ValueVector to) { return new TransferImpl((${minor.class}Vector) to); } - public void transferTo(${minor.class}Vector target){ + public void transferTo(${minor.class}Vector target) { target.clear(); this.offsetVector.transferTo(target.offsetVector); target.data = data; - target.data.retain(); + target.data.retain(1); clear(); } public void splitAndTransferTo(int startIndex, int length, ${minor.class}Vector target) { - int startPoint = this.offsetVector.getAccessor().get(startIndex); - int sliceLength = this.offsetVector.getAccessor().get(startIndex + length) - startPoint; - target.offsetVector.clear(); + UInt${type.width}Vector.Accessor offsetVectorAccessor = this.offsetVector.getAccessor(); + final int startPoint = offsetVectorAccessor.get(startIndex); + final int sliceLength = offsetVectorAccessor.get(startIndex + length) - startPoint; + target.clear(); target.offsetVector.allocateNew(length + 1); + offsetVectorAccessor = this.offsetVector.getAccessor(); + final UInt4Vector.Mutator offsetVectorMutator = target.offsetVector.getMutator(); for (int i = 0; i < length + 1; i++) { - target.offsetVector.getMutator().set(i, this.offsetVector.getAccessor().get(startIndex + i) - startPoint); + offsetVectorMutator.set(i, offsetVectorAccessor.get(startIndex + i) - startPoint); } - target.data = this.data.slice(startPoint, sliceLength); - target.data.retain(); + target.data = data.slice(startPoint, sliceLength); + target.data.retain(1); target.getMutator().setValueCount(length); } - protected void copyFrom(int fromIndex, int thisIndex, ${minor.class}Vector from){ - int start = from.offsetVector.getAccessor().get(fromIndex); - int end = from.offsetVector.getAccessor().get(fromIndex+1); - int len = end - start; + protected void copyFrom(int fromIndex, int thisIndex, ${minor.class}Vector from) { + final UInt4Vector.Accessor fromOffsetVectorAccessor = from.offsetVector.getAccessor(); + final int start = fromOffsetVectorAccessor.get(fromIndex); + final int end = fromOffsetVectorAccessor.get(fromIndex + 1); + final int len = end - start; - int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width}); + final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width}); from.data.getBytes(start, data, outputStart, len); offsetVector.data.set${(minor.javaType!type.javaType)?cap_first}( (thisIndex+1) * ${type.width}, outputStart + len); } - public boolean copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from){ - - int start = from.offsetVector.getAccessor().get(fromIndex); - int end = from.offsetVector.getAccessor().get(fromIndex+1); - int len = end - start; - - int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width}); + public boolean copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from) { + final UInt${type.width}Vector.Accessor fromOffsetVectorAccessor = from.offsetVector.getAccessor(); + final int start = fromOffsetVectorAccessor.get(fromIndex); + final int end = fromOffsetVectorAccessor.get(fromIndex + 1); + final int len = end - start; + final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width}); while(data.capacity() < outputStart + len) { - reAlloc(); + reAlloc(); } offsetVector.getMutator().setSafe(thisIndex + 1, outputStart + len); - from.data.getBytes(start, data, outputStart, len); - return true; } - - private class TransferImpl implements TransferPair{ + private class TransferImpl implements TransferPair { ${minor.class}Vector to; - public TransferImpl(MaterializedField field){ - this.to = new ${minor.class}Vector(field, allocator); + public TransferImpl(MaterializedField field) { + to = new ${minor.class}Vector(field, allocator); } - public TransferImpl(${minor.class}Vector to){ + public TransferImpl(${minor.class}Vector to) { this.to = to; } - public ${minor.class}Vector getTo(){ + @Override + public ${minor.class}Vector getTo() { return to; } - public void transfer(){ + @Override + public void transfer() { transferTo(to); } + @Override public void splitAndTransfer(int startIndex, int length) { splitAndTransferTo(startIndex, length, to); } @@ -273,8 +281,9 @@ public void setInitialCapacity(final int valueCount) { offsetVector.setInitialCapacity(valueCount + 1); } + @Override public void allocateNew() { - if(!allocateNewSafe()){ + if(!allocateNewSafe()) { throw new OutOfMemoryRuntimeException("Failure while allocating buffer."); } } @@ -301,7 +310,7 @@ public boolean allocateNewSafe() { * clear all the memory that we allocated */ try { - final int requestedSize = (int)curAllocationSize; + final int requestedSize = (int) curAllocationSize; data = allocator.buffer(requestedSize); offsetVector.allocateNew(); } catch (OutOfMemoryRuntimeException e) { @@ -313,6 +322,7 @@ public boolean allocateNewSafe() { return true; } + @Override public void allocateNew(int totalBytes, int valueCount) { clear(); assert totalBytes >= 0; @@ -352,42 +362,45 @@ private void incrementAllocationMonitor() { ++allocationMonitor; } - public Accessor getAccessor(){ + @Override + public Accessor getAccessor() { return accessor; } + @Override public Mutator getMutator() { return mutator; } public final class Accessor extends BaseValueVector.BaseAccessor implements VariableWidthAccessor { final UInt${type.width}Vector.Accessor oAccessor = offsetVector.getAccessor(); - - public long getStartEnd(int index){ + public long getStartEnd(int index) { return oAccessor.getTwoAsLong(index); } public byte[] get(int index) { assert index >= 0; - int startIdx = oAccessor.get(index); - int length = oAccessor.get(index + 1) - startIdx; + final int startIdx = oAccessor.get(index); + final int length = oAccessor.get(index + 1) - startIdx; assert length >= 0; - byte[] dst = new byte[length]; + final byte[] dst = new byte[length]; data.getBytes(startIdx, dst, 0, length); return dst; } + @Override public int getValueLength(int index) { - return offsetVector.getAccessor().get(index + 1) - offsetVector.getAccessor().get(index); + final UInt${type.width}Vector.Accessor offsetVectorAccessor = offsetVector.getAccessor(); + return offsetVectorAccessor.get(index + 1) - offsetVectorAccessor.get(index); } - public void get(int index, ${minor.class}Holder holder){ + public void get(int index, ${minor.class}Holder holder) { holder.start = oAccessor.get(index); holder.end = oAccessor.get(index + 1); holder.buffer = data; } - public void get(int index, Nullable${minor.class}Holder holder){ + public void get(int index, Nullable${minor.class}Holder holder) { holder.isSet = 1; holder.start = oAccessor.get(index); holder.end = oAccessor.get(index + 1); @@ -397,6 +410,7 @@ public void get(int index, Nullable${minor.class}Holder holder){ <#switch minor.class> <#case "VarChar"> + @Override public ${friendlyType} getObject(int index) { Text text = new Text(); text.set(get(index)); @@ -404,28 +418,29 @@ public void get(int index, Nullable${minor.class}Holder holder){ } <#break> <#case "Var16Char"> + @Override public ${friendlyType} getObject(int index) { return new String(get(index), Charsets.UTF_16); } <#break> <#default> + @Override public ${friendlyType} getObject(int index) { return get(index); } - - - + @Override public int getValueCount() { return Math.max(offsetVector.getAccessor().getValueCount()-1, 0); } - public boolean isNull(int index){ + @Override + public boolean isNull(int index) { return false; } - public UInt${type.width}Vector getOffsetVector(){ + public UInt${type.width}Vector getOffsetVector() { return offsetVector; } } @@ -450,7 +465,7 @@ public final class Mutator extends BaseValueVector.BaseMutator implements Variab */ protected void set(int index, byte[] bytes) { assert index >= 0; - int currentOffset = offsetVector.getAccessor().get(index); + final int currentOffset = offsetVector.getAccessor().get(index); offsetVector.getMutator().set(index + 1, currentOffset + bytes.length); data.setBytes(currentOffset, bytes, 0, bytes.length); } @@ -458,12 +473,11 @@ protected void set(int index, byte[] bytes) { public void setSafe(int index, byte[] bytes) { assert index >= 0; - int currentOffset = offsetVector.getAccessor().get(index); - while (data.capacity() < currentOffset + bytes.length) { + final int currentOffset = offsetVector.getAccessor().get(index); + while(data.capacity() < currentOffset + bytes.length) { reAlloc(); } offsetVector.getMutator().setSafe(index + 1, currentOffset + bytes.length); - offsetVector.getMutator().set(index + 1, currentOffset + bytes.length); data.setBytes(currentOffset, bytes, 0, bytes.length); } @@ -477,7 +491,7 @@ public void setSafe(int index, byte[] bytes) { */ protected void set(int index, byte[] bytes, int start, int length) { assert index >= 0; - int currentOffset = offsetVector.getAccessor().get(index); + final int currentOffset = offsetVector.getAccessor().get(index); offsetVector.getMutator().set(index + 1, currentOffset + length); data.setBytes(currentOffset, bytes, start, length); } @@ -497,7 +511,7 @@ public void setSafe(int index, ByteBuffer bytes, int start, int length) { public void setSafe(int index, byte[] bytes, int start, int length) { assert index >= 0; - int currentOffset = offsetVector.getAccessor().get(index); + final int currentOffset = offsetVector.getAccessor().get(index); while (data.capacity() < currentOffset + length) { reAlloc(); @@ -506,8 +520,9 @@ public void setSafe(int index, byte[] bytes, int start, int length) { data.setBytes(currentOffset, bytes, start, length); } + @Override public void setValueLengthSafe(int index, int length) { - int offset = offsetVector.getAccessor().get(index); + final int offset = offsetVector.getAccessor().get(index); while(data.capacity() < offset + length ) { reAlloc(); } @@ -515,10 +530,9 @@ public void setValueLengthSafe(int index, int length) { } - public void setSafe(int index, int start, int end, DrillBuf buffer){ - int len = end - start; - - int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}); + public void setSafe(int index, int start, int end, DrillBuf buffer) { + final int len = end - start; + final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}); while(data.capacity() < outputStart + len) { reAlloc(); @@ -528,13 +542,12 @@ public void setSafe(int index, int start, int end, DrillBuf buffer){ buffer.getBytes(start, data, outputStart, len); } - - public void setSafe(int index, Nullable${minor.class}Holder holder){ + public void setSafe(int index, Nullable${minor.class}Holder holder) { assert holder.isSet == 1; - int start = holder.start; - int end = holder.end; - int len = end - start; + final int start = holder.start; + final int end = holder.end; + final int len = end - start; int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}); @@ -546,13 +559,11 @@ public void setSafe(int index, Nullable${minor.class}Holder holder){ offsetVector.getMutator().setSafe( index+1, outputStart + len); } - public void setSafe(int index, ${minor.class}Holder holder){ - - int start = holder.start; - int end = holder.end; - int len = end - start; - - int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}); + public void setSafe(int index, ${minor.class}Holder holder) { + final int start = holder.start; + final int end = holder.end; + final int len = end - start; + final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}); while(data.capacity() < outputStart + len) { reAlloc(); @@ -562,31 +573,32 @@ public void setSafe(int index, ${minor.class}Holder holder){ offsetVector.getMutator().setSafe( index+1, outputStart + len); } - protected void set(int index, int start, int length, DrillBuf buffer){ + protected void set(int index, int start, int length, DrillBuf buffer) { assert index >= 0; - int currentOffset = offsetVector.getAccessor().get(index); + final int currentOffset = offsetVector.getAccessor().get(index); offsetVector.getMutator().set(index + 1, currentOffset + length); - DrillBuf bb = buffer.slice(start, length); + final DrillBuf bb = buffer.slice(start, length); data.setBytes(currentOffset, bb); } - protected void set(int index, Nullable${minor.class}Holder holder){ - int length = holder.end - holder.start; - int currentOffset = offsetVector.getAccessor().get(index); + protected void set(int index, Nullable${minor.class}Holder holder) { + final int length = holder.end - holder.start; + final int currentOffset = offsetVector.getAccessor().get(index); offsetVector.getMutator().set(index + 1, currentOffset + length); data.setBytes(currentOffset, holder.buffer, holder.start, length); } - protected void set(int index, ${minor.class}Holder holder){ - int length = holder.end - holder.start; - int currentOffset = offsetVector.getAccessor().get(index); + protected void set(int index, ${minor.class}Holder holder) { + final int length = holder.end - holder.start; + final int currentOffset = offsetVector.getAccessor().get(index); offsetVector.getMutator().set(index + 1, currentOffset + length); data.setBytes(currentOffset, holder.buffer, holder.start, length); } + @Override public void setValueCount(int valueCount) { - int currentByteCapacity = getByteCapacity(); - int idx = offsetVector.getAccessor().get(valueCount); + final int currentByteCapacity = getByteCapacity(); + final int idx = offsetVector.getAccessor().get(valueCount); data.writerIndex(idx); if (valueCount > 0 && currentByteCapacity > idx * 2) { incrementAllocationMonitor(); @@ -598,30 +610,26 @@ public void setValueCount(int valueCount) { } @Override - public void generateTestData(int size){ + public void generateTestData(int size) { boolean even = true; <#switch minor.class> <#case "Var16Char"> - java.nio.charset.Charset charset = Charsets.UTF_16; + final java.nio.charset.Charset charset = Charsets.UTF_16; <#break> <#case "VarChar"> <#default> - java.nio.charset.Charset charset = Charsets.UTF_8; + final java.nio.charset.Charset charset = Charsets.UTF_8; - for(int i =0; i < size; i++, even = !even){ - if(even){ - set(i, new String("aaaaa").getBytes(charset)); - }else{ - set(i, new String("bbbbbbbbbb").getBytes(charset)); - } + final byte[] evenValue = new String("aaaaa").getBytes(charset); + final byte[] oddValue = new String("bbbbbbbbbb").getBytes(charset); + for(int i =0; i < size; i++, even = !even) { + set(i, even ? evenValue : oddValue); } setValueCount(size); } } - } - <#-- type.major --> diff --git a/exec/java-exec/src/main/java/io/netty/buffer/DrillBuf.java b/exec/java-exec/src/main/java/io/netty/buffer/DrillBuf.java index 3ec6b3e0fe7..14f44c20fb2 100644 --- a/exec/java-exec/src/main/java/io/netty/buffer/DrillBuf.java +++ b/exec/java-exec/src/main/java/io/netty/buffer/DrillBuf.java @@ -27,30 +27,39 @@ import java.nio.channels.GatheringByteChannel; import java.nio.channels.ScatteringByteChannel; import java.nio.charset.Charset; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.drill.exec.memory.Accountor; +import java.util.Collection; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.LinkedList; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.drill.common.HistoricalLog; +import org.apache.drill.exec.memory.BaseAllocator; import org.apache.drill.exec.memory.BufferAllocator; import org.apache.drill.exec.ops.BufferManager; +import org.apache.drill.exec.memory.BufferLedger; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.ops.OperatorContext; import org.apache.drill.exec.util.AssertionUtil; +import org.apache.drill.exec.util.Pointer; +import org.slf4j.Logger; +import com.google.common.base.Charsets; import com.google.common.base.Preconditions; public final class DrillBuf extends AbstractByteBuf implements AutoCloseable { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillBuf.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillBuf.class); private static final boolean BOUNDS_CHECKING_ENABLED = AssertionUtil.BOUNDS_CHECKING_ENABLED; + private static final boolean DEBUG = BaseAllocator.isDebug(); + private static final AtomicInteger idGenerator = new AtomicInteger(0); - private final ByteBuf b; + private final ByteBuf byteBuf; private final long addr; private final int offset; - private final boolean rootBuffer; - private final AtomicLong rootRefCnt = new AtomicLong(1); + private final int flags; + private final AtomicInteger rootRefCnt; private volatile BufferAllocator allocator; - private volatile Accountor acct; - private volatile int length; // TODO - cleanup // The code is partly shared and partly copy-pasted between @@ -58,83 +67,227 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable { // to share code and to remove the hacky code here to use only // one of these types at a time and use null checks to find out // which. - private OperatorContext context; - private FragmentContext fContext; private BufferManager bufManager; + @Deprecated private OperatorContext operatorContext; + @Deprecated private FragmentContext fragmentContext; + + private volatile BufferLedger bufferLedger; + private volatile int length; // TODO this just seems to duplicate .capacity() + + // members used purely for debugging + // TODO(cwestin) once we have a reduced number of constructors, move these to DEBUG clauses in them + private final int id = idGenerator.incrementAndGet(); + private final HistoricalLog historicalLog = DEBUG ? new HistoricalLog(4, "DrillBuf[%d]", id) : null; + private final static IdentityHashMap> unwrappedMap = + DEBUG ? new IdentityHashMap>() : null; + + // TODO(cwestin) javadoc + private void unwrappedPut() { + final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) byteBuf; + synchronized(unwrappedMap) { + Collection drillBufs = unwrappedMap.get(udle); + if (drillBufs == null) { + drillBufs = new LinkedList(); + unwrappedMap.put(udle, drillBufs); + } - public DrillBuf(BufferAllocator allocator, Accountor a, UnsafeDirectLittleEndian b) { - super(b.maxCapacity()); - this.b = b; - this.addr = b.memoryAddress(); - this.acct = a; - this.length = b.capacity(); - this.offset = 0; - this.rootBuffer = true; - this.allocator = allocator; + drillBufs.add(this); + } } - private DrillBuf(BufferAllocator allocator, Accountor a) { + // TODO(cwestin) javadoc + public static Collection unwrappedGet(final UnsafeDirectLittleEndian udle) { + synchronized(unwrappedMap) { + final Collection drillBufs = unwrappedMap.get(udle); + if (drillBufs == null) { + return Collections.emptyList(); + } + return new LinkedList(drillBufs); + } + } + + // TODO(cwestin) javadoc + private static boolean unwrappedRemove(final DrillBuf drillBuf) { + final ByteBuf byteBuf = drillBuf.unwrap(); + if (!(byteBuf instanceof UnsafeDirectLittleEndian)) { + return false; + } + + final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) byteBuf; + synchronized(unwrappedMap) { + Collection drillBufs = unwrappedMap.get(udle); + if (drillBufs == null) { + return false; + } + final Object object = drillBufs.remove(drillBuf); + if (drillBufs.isEmpty()) { + unwrappedMap.remove(udle); + } + return object != null; + } + } + + // TODO(cwestin) javadoc + public DrillBuf(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator, + final UnsafeDirectLittleEndian byteBuf) { + super(byteBuf.maxCapacity()); + this.byteBuf = byteBuf; + byteBuf.retain(1); + this.bufferLedger = bufferLedger; + addr = byteBuf.memoryAddress(); + allocator = bufferAllocator; + length = byteBuf.capacity(); + offset = 0; + flags = 0; + rootRefCnt = new AtomicInteger(1); + + if (DEBUG) { + unwrappedPut(); + historicalLog.recordEvent( + "DrillBuf(BufferLedger, BufferAllocator[%d], UnsafeDirectLittleEndian[identityHashCode == " + + "%d](%s)) => rootRefCnt identityHashCode == %d", + bufferAllocator.getId(), System.identityHashCode(byteBuf), byteBuf.toString(), + System.identityHashCode(rootRefCnt)); + } + } + + private DrillBuf(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator) { super(0); - this.b = new EmptyByteBuf(allocator.getUnderlyingAllocator()).order(ByteOrder.LITTLE_ENDIAN); - this.allocator = allocator; - this.acct = a; - this.length = 0; - this.addr = 0; - this.rootBuffer = false; - this.offset = 0; + this.bufferLedger = bufferLedger; + allocator = bufferAllocator; + + byteBuf = new EmptyByteBuf(bufferLedger.getUnderlyingAllocator()).order(ByteOrder.LITTLE_ENDIAN); + length = 0; + addr = 0; + flags = 0; + rootRefCnt = new AtomicInteger(1); + offset = 0; + + if (DEBUG) { + // We don't put the empty buffers in the unwrappedMap. + historicalLog.recordEvent( + "DrillBuf(BufferLedger, BufferAllocator[%d]) => rootRefCnt identityHashCode == %d", + bufferAllocator.getId(), System.identityHashCode(rootRefCnt)); + } } + public static DrillBuf getEmpty(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator) { + return new DrillBuf(bufferLedger, bufferAllocator); + } + + /** + * Indicate a shared refcount, as per http://netty.io/wiki/reference-counted-objects.html#wiki-h3-5 + */ + private final static int F_DERIVED = 0x0002; + + // TODO(cwestin) javadoc /** - * Special constructor used for RPC ownership transfer. Takes a snapshot slice of the current buf - * but points directly to the underlying UnsafeLittleEndian buffer. Does this by calling unwrap() - * twice on the provided DrillBuf and expecting an UnsafeDirectLittleEndian buffer. This operation - * includes taking a new reference count on the underlying buffer and maintaining returning with a - * current reference count for itself (masking the underlying reference count). - * @param allocator - * @param a Allocator used when users try to receive allocator from buffer. - * @param b Accountor used for accounting purposes. + * Used for sharing. + * + * @param bufferLedger + * @param bufferAllocator + * @param originalBuf + * @param index + * @param length + * @param flags */ - public DrillBuf(BufferAllocator allocator, Accountor a, DrillBuf b) { - this(allocator, a, getUnderlying(b), b, 0, b.length, true); - assert b.unwrap().unwrap() instanceof UnsafeDirectLittleEndian; - b.unwrap().unwrap().retain(); + public DrillBuf(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator, + final DrillBuf originalBuf, final int index, final int length, final int flags) { + this(bufferAllocator, bufferLedger, getUnderlyingUdle(originalBuf), + originalBuf, index + originalBuf.offset, length, flags); } + /** + * Unwraps a DrillBuf until the underlying UnsafeDirectLittleEndian buffer is + * found. + * + * @param originalBuf the original DrillBuf + * @return the underlying UnsafeDirectLittleEndian ByteBuf + */ + private static ByteBuf getUnderlyingUdle(final DrillBuf originalBuf) { + int count = 1; + ByteBuf unwrapped = originalBuf.unwrap(); + while(!(unwrapped instanceof UnsafeDirectLittleEndian) + && (!(unwrapped instanceof EmptyByteBuf))) { + unwrapped = unwrapped.unwrap(); + ++count; + } - private DrillBuf(DrillBuf buffer, int index, int length) { - this(buffer.allocator, null, buffer, buffer, index, length, false); - } + if (DEBUG) { + if (count > 1) { + throw new IllegalStateException("UnsafeDirectLittleEndian is wrapped more than one level"); + } + } - private static ByteBuf getUnderlying(DrillBuf b){ - ByteBuf underlying = b.unwrap().unwrap(); - return underlying.slice((int) (b.memoryAddress() - underlying.memoryAddress()), b.length); + return unwrapped; } - private DrillBuf(BufferAllocator allocator, Accountor a, ByteBuf replacement, DrillBuf buffer, int index, int length, boolean root) { - super(length); - if (index < 0 || index > buffer.capacity() - length) { - throw new IndexOutOfBoundsException(buffer.toString() + ".slice(" + index + ", " + length + ')'); + + // TODO(cwestin) javadoc + /* + * TODO(cwestin) the replacement argument becomes an UnsafeDirectLittleEndian; + * buffer argument may go away if it is determined to be unnecessary after all + * the deprecated stuff is removed (I suspect only the replacement argument is + * necessary then). + */ + private DrillBuf(BufferAllocator allocator, BufferLedger bufferLedger, + ByteBuf replacement, DrillBuf buffer, int index, int length, int flags) { + super(replacement.maxCapacity()); + + if (index < 0 || index > (replacement.maxCapacity() - length)) { + throw new IndexOutOfBoundsException(replacement.toString() + ".slice(" + index + ", " + length + ')'); } - this.length = length; + this.flags = flags; + + this.length = length; // capacity() writerIndex(length); - this.b = replacement; - this.addr = buffer.memoryAddress() + index; - this.offset = index; - this.acct = a; - this.length = length; - this.rootBuffer = root; + byteBuf = replacement; + if ((flags & F_DERIVED) == 0) { + replacement.retain(1); + } + + addr = replacement.memoryAddress() + index; + offset = index; + this.bufferLedger = bufferLedger; + if (!(buffer instanceof DrillBuf)) { + throw new IllegalArgumentException("DrillBuf slicing can only be performed on other DrillBufs"); + } + + if ((flags & F_DERIVED) != 0) { + final DrillBuf rootBuf = (DrillBuf) buffer; + rootRefCnt = rootBuf.rootRefCnt; + } else { + rootRefCnt = new AtomicInteger(1); + } + this.allocator = allocator; + + if (DEBUG) { + unwrappedPut(); + historicalLog.recordEvent( + "DrillBuf(BufferAllocator[%d], BufferLedger, ByteBuf[identityHashCode == " + + "%d](%s), DrillBuf[%d], index = %d, length = %d, flags = 0x%08x)" + + " => rootRefCnt identityHashCode == %d", + allocator.getId(), System.identityHashCode(replacement), replacement.toString(), + buffer.id, index, length, flags, System.identityHashCode(rootRefCnt)); + } } + @Deprecated public void setOperatorContext(OperatorContext c) { - this.context = c; + this.operatorContext = c; } + + @Deprecated public void setFragmentContext(FragmentContext c) { - this.fContext = c; + this.fragmentContext = c; } + // TODO(DRILL-3331) public void setBufferManager(BufferManager bufManager) { + Preconditions.checkState(this.bufManager == null, + "the BufferManager for a buffer can only be set once"); this.bufManager = bufManager; } @@ -142,50 +295,42 @@ public BufferAllocator getAllocator() { return allocator; } - public DrillBuf reallocIfNeeded(int size) { + public DrillBuf reallocIfNeeded(final int size) { + Preconditions.checkArgument(size >= 0, "reallocation size must be non-negative"); + if (this.capacity() >= size) { return this; } - if (context != null) { - return context.replace(this, size); - } else if(fContext != null) { - return fContext.replace(this, size); + + if (operatorContext != null) { + return operatorContext.replace(this, size); + } else if(fragmentContext != null) { + return fragmentContext.replace(this, size); } else if (bufManager != null) { return bufManager.replace(this, size); } else { throw new UnsupportedOperationException("Realloc is only available in the context of an operator's UDFs"); } - } @Override public int refCnt() { - if(rootBuffer){ - return (int) this.rootRefCnt.get(); - }else{ - return b.refCnt(); - } - + return rootRefCnt.get(); } private long addr(int index) { return addr + index; } - private final void checkIndexD(int index) { - ensureAccessible(); - if (index < 0 || index >= capacity()) { - throw new IndexOutOfBoundsException(String.format( - "index: %d (expected: range(0, %d))", index, capacity())); - } - } - private final void checkIndexD(int index, int fieldLength) { ensureAccessible(); if (fieldLength < 0) { throw new IllegalArgumentException("length: " + fieldLength + " (expected: >= 0)"); } if (index < 0 || index > capacity() - fieldLength) { + if (DEBUG) { + historicalLog.logHistory(logger); + } throw new IndexOutOfBoundsException(String.format( "index: %d, length: %d (expected: range(0, %d))", index, fieldLength, capacity())); } @@ -200,7 +345,7 @@ private final void checkIndexD(int index, int fieldLength) { * @param start The starting position of the bytes to be read. * @param end The exclusive endpoint of the bytes to be read. */ - public void checkBytes(int start, int end){ + public void checkBytes(int start, int end) { if (BOUNDS_CHECKING_ENABLED) { checkIndexD(start, end - start); } @@ -212,30 +357,53 @@ private void chk(int index, int width) { } } - private void chk(int index) { - if (BOUNDS_CHECKING_ENABLED) { - checkIndexD(index); - } - } - private void ensure(int width) { if (BOUNDS_CHECKING_ENABLED) { ensureWritable(width); } } - public boolean transferAccounting(Accountor target) { - if (rootBuffer) { - boolean outcome = acct.transferTo(target, this, length); - acct = target; - return outcome; - } else { - throw new UnsupportedOperationException(); - } + /** + * Used by allocators to transfer ownership from one allocator to another. + * + * @param newLedger the new ledger the buffer should use going forward + * @param newAllocator the new allocator + * @return whether or not the buffer fits the receiving allocator's allocation limit + */ + public boolean transferTo(final BufferAllocator newAllocator, final BufferLedger newLedger) { + final Pointer pNewLedger = new Pointer<>(newLedger); + final boolean fitsAllocation = bufferLedger.transferTo(newAllocator, pNewLedger, this); + allocator = newAllocator; + bufferLedger = pNewLedger.value; + return fitsAllocation; + } + + /** + * DrillBuf's implementation of sharing buffer functionality, to be accessed from + * {@link BufferAllocator#shareOwnership(DrillBuf, Pointer)}. See that function + * for more information. + * + * @param otherLedger the ledger belonging to the other allocator to share with + * @param otherAllocator the other allocator to be shared with + * @param index the starting index (for slicing capability) + * @param length the length (for slicing capability) + * @return the new DrillBuf (wrapper) + */ + public DrillBuf shareWith(final BufferLedger otherLedger, final BufferAllocator otherAllocator, + final int index, final int length) { + return shareWith(otherLedger, otherAllocator, index, length, 0); + } + + // TODO(cwestin) javadoc + private DrillBuf shareWith(final BufferLedger otherLedger, final BufferAllocator otherAllocator, + final int index, final int length, final int flags) { + final Pointer pDrillBuf = new Pointer<>(); + bufferLedger = bufferLedger.shareWith(pDrillBuf, otherLedger, otherAllocator, this, index, length, flags); + return pDrillBuf.value; } @Override - public synchronized boolean release() { + public boolean release() { return release(1); } @@ -244,20 +412,28 @@ public synchronized boolean release() { */ @Override public synchronized boolean release(int decrement) { + Preconditions.checkArgument(decrement > 0, + "release(%d) argument is not positive", decrement); + if (DEBUG) { + historicalLog.recordEvent("release(%d)", decrement); + } - if(rootBuffer){ - final long newRefCnt = this.rootRefCnt.addAndGet(-decrement); - Preconditions.checkArgument(newRefCnt > -1, "Buffer has negative reference count."); - if (newRefCnt == 0) { - b.release(decrement); - acct.release(this, length); - return true; - }else{ - return false; + final int refCnt = rootRefCnt.addAndGet(-decrement); + Preconditions.checkState(refCnt >= 0, "DrillBuf[%d] refCnt has gone negative", id); + if (refCnt == 0) { + bufferLedger.release(this); + + if (DEBUG) { + unwrappedRemove(this); } - }else{ - return b.release(decrement); + + // release the underlying buffer + byteBuf.release(1); + + return true; } + + return false; } @Override @@ -267,31 +443,27 @@ public int capacity() { @Override public synchronized ByteBuf capacity(int newCapacity) { - if (rootBuffer) { - if (newCapacity == length) { - return this; - } else if (newCapacity < length) { - b.capacity(newCapacity); - int diff = length - b.capacity(); - acct.releasePartial(this, diff); - this.length = length - diff; - return this; - } else { - throw new UnsupportedOperationException("Accounting byte buf doesn't support increasing allocations."); - } - } else { - throw new UnsupportedOperationException("Non root bufs doen't support changing allocations."); + if ((flags & F_DERIVED) != 0) { + throw new UnsupportedOperationException("Derived buffers don't support resizing."); } - } - @Override - public int maxCapacity() { - return length; + if (newCapacity == length) { + return this; + } + + if (newCapacity < length) { + byteBuf.capacity(newCapacity); + final int diff = length - byteBuf.capacity(); + length -= diff; + return this; + } + + throw new UnsupportedOperationException("Buffers don't support resizing that increases the size."); } @Override public ByteBufAllocator alloc() { - return b.alloc(); + return byteBuf.alloc(); } @Override @@ -301,14 +473,12 @@ public ByteOrder order() { @Override public ByteBuf order(ByteOrder endianness) { - // if(endianness != ByteOrder.LITTLE_ENDIAN) throw new - // UnsupportedOperationException("Drill buffers only support little endian."); return this; } @Override public ByteBuf unwrap() { - return b; + return byteBuf; } @Override @@ -323,7 +493,7 @@ public ByteBuf readBytes(int length) { @Override public ByteBuf readSlice(int length) { - ByteBuf slice = slice(readerIndex(), length); + final ByteBuf slice = slice(readerIndex(), length); readerIndex(readerIndex() + length); return slice; } @@ -343,16 +513,32 @@ public ByteBuf slice() { return slice(readerIndex(), readableBytes()); } + public static String bufferState(final ByteBuf buf) { + final int cap = buf.capacity(); + final int mcap = buf.maxCapacity(); + final int ri = buf.readerIndex(); + final int rb = buf.readableBytes(); + final int wi = buf.writerIndex(); + final int wb = buf.writableBytes(); + return String.format("cap/max: %d/%d, ri: %d, rb: %d, wi: %d, wb: %d", + cap, mcap, ri, rb, wi, wb); + } + @Override public DrillBuf slice(int index, int length) { - DrillBuf buf = new DrillBuf(this, index, length); - buf.writerIndex = length; + /* + * Re the behavior of reference counting, + * see http://netty.io/wiki/reference-counted-objects.html#wiki-h3-5, which explains + * that derived buffers share their reference count with their parent + */ + final DrillBuf buf = shareWith(bufferLedger, allocator, index, length, F_DERIVED); + buf.writerIndex(length); return buf; } @Override public DrillBuf duplicate() { - return new DrillBuf(this, 0, length); + return slice(0, length); } @Override @@ -367,12 +553,12 @@ public ByteBuffer nioBuffer() { @Override public ByteBuffer nioBuffer(int index, int length) { - return b.nioBuffer(offset + index, length); + return byteBuf.nioBuffer(offset + index, length); } @Override public ByteBuffer internalNioBuffer(int index, int length) { - return b.internalNioBuffer(offset + index, length); + return byteBuf.internalNioBuffer(offset + index, length); } @Override @@ -387,17 +573,17 @@ public ByteBuffer[] nioBuffers(int index, int length) { @Override public boolean hasArray() { - return b.hasArray(); + return byteBuf.hasArray(); } @Override public byte[] array() { - return b.array(); + return byteBuf.array(); } @Override public int arrayOffset() { - return b.arrayOffset(); + return byteBuf.arrayOffset(); } @Override @@ -410,18 +596,27 @@ public long memoryAddress() { return this.addr; } + @Override + public String toString() { + return toString(0, 0, Charsets.UTF_8); + } + @Override public String toString(Charset charset) { - return toString(readerIndex, readableBytes(), charset); + return toString(readerIndex, readableBytes(), charset); } @Override public String toString(int index, int length, Charset charset) { + final String basics = + String.format("{DrillBuf[%d], udle identityHashCode == %d, rootRefCnt identityHashCode == %d}", + id, System.identityHashCode(byteBuf), System.identityHashCode(rootRefCnt)); + if (length == 0) { - return ""; + return basics; } - ByteBuffer nioBuffer; + final ByteBuffer nioBuffer; if (nioBufferCount() == 1) { nioBuffer = nioBuffer(index, length); } else { @@ -430,7 +625,7 @@ public String toString(int index, int length, Charset charset) { nioBuffer.flip(); } - return ByteBufUtil.decodeString(nioBuffer, charset); + return basics + '\n' + ByteBufUtil.decodeString(nioBuffer, charset); } @Override @@ -446,11 +641,12 @@ public boolean equals(Object obj) { @Override public ByteBuf retain(int increment) { - if(rootBuffer){ - this.rootRefCnt.addAndGet(increment); - }else{ - b.retain(increment); + Preconditions.checkArgument(increment > 0, "retain(%d) argument is not positive", increment); + if (DEBUG) { + historicalLog.recordEvent("retain(%d)", increment); } + + rootRefCnt.addAndGet(increment); return this; } @@ -462,7 +658,7 @@ public ByteBuf retain() { @Override public long getLong(int index) { chk(index, 8); - long v = PlatformDependent.getLong(addr(index)); + final long v = PlatformDependent.getLong(addr(index)); return v; } @@ -489,7 +685,7 @@ public long getUnsignedInt(int index) { @Override public int getInt(int index) { chk(index, 4); - int v = PlatformDependent.getInt(addr(index)); + final int v = PlatformDependent.getInt(addr(index)); return v; } @@ -597,13 +793,13 @@ public ByteBuf writeDouble(double value) { @Override public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length) { - b.getBytes(index + offset, dst, dstIndex, length); + byteBuf.getBytes(index + offset, dst, dstIndex, length); return this; } @Override public ByteBuf getBytes(int index, ByteBuffer dst) { - b.getBytes(index + offset, dst); + byteBuf.getBytes(index + offset, dst); return this; } @@ -671,19 +867,19 @@ protected void _setLong(int index, long value) { @Override public ByteBuf getBytes(int index, ByteBuf dst, int dstIndex, int length) { - b.getBytes(index + offset, dst, dstIndex, length); + byteBuf.getBytes(index + offset, dst, dstIndex, length); return this; } @Override public ByteBuf getBytes(int index, OutputStream out, int length) throws IOException { - b.getBytes(index + offset, out, length); + byteBuf.getBytes(index + offset, out, length); return this; } @Override protected int _getUnsignedMedium(int index) { - long addr = addr(index); + final long addr = addr(index); return (PlatformDependent.getByte(addr) & 0xff) << 16 | (PlatformDependent.getByte(addr + 1) & 0xff) << 8 | PlatformDependent.getByte(addr + 2) & 0xff; @@ -691,12 +887,12 @@ protected int _getUnsignedMedium(int index) { @Override public int getBytes(int index, GatheringByteChannel out, int length) throws IOException { - return b.getBytes(index + offset, out, length); + return byteBuf.getBytes(index + offset, out, length); } @Override public ByteBuf setBytes(int index, ByteBuf src, int srcIndex, int length) { - b.setBytes(index + offset, src, srcIndex, length); + byteBuf.setBytes(index + offset, src, srcIndex, length); return this; } @@ -707,12 +903,12 @@ public ByteBuf setBytes(int index, ByteBuffer src, int srcIndex, int length) { length); } else { if (srcIndex == 0 && src.capacity() == length) { - b.setBytes(index + offset, src); + byteBuf.setBytes(index + offset, src); } else { ByteBuffer newBuf = src.duplicate(); newBuf.position(srcIndex); newBuf.limit(srcIndex + length); - b.setBytes(index + offset, src); + byteBuf.setBytes(index + offset, src); } } @@ -721,24 +917,24 @@ public ByteBuf setBytes(int index, ByteBuffer src, int srcIndex, int length) { @Override public ByteBuf setBytes(int index, byte[] src, int srcIndex, int length) { - b.setBytes(index + offset, src, srcIndex, length); + byteBuf.setBytes(index + offset, src, srcIndex, length); return this; } @Override public ByteBuf setBytes(int index, ByteBuffer src) { - b.setBytes(index + offset, src); + byteBuf.setBytes(index + offset, src); return this; } @Override public int setBytes(int index, InputStream in, int length) throws IOException { - return b.setBytes(index + offset, in, length); + return byteBuf.setBytes(index + offset, in, length); } @Override public int setBytes(int index, ScatteringByteChannel in, int length) throws IOException { - return b.setBytes(index + offset, in, length); + return byteBuf.setBytes(index + offset, in, length); } @Override @@ -747,12 +943,25 @@ public byte getByte(int index) { return PlatformDependent.getByte(addr(index)); } - public static DrillBuf getEmpty(BufferAllocator allocator, Accountor a) { - return new DrillBuf(allocator, a); - } - + /** + * Find out if this is a "root buffer." This is obsolete terminology + * based on the original implementation of DrillBuf, which would layer + * DrillBufs on top of other DrillBufs when slicing (or duplicating). + * The buffer at the bottom of the layer was the "root buffer." However, + * the current implementation flattens such references to always make + * DrillBufs that are wrap a single buffer underneath, and slices and + * their original source have a shared fate as per + * http://netty.io/wiki/reference-counted-objects.html#wiki-h3-5, so + * this concept isn't really meaningful anymore. But there are callers + * that want to know a buffer's original size, and whether or not it + * is "primal" in some sense. Perhaps this just needs a new name that + * indicates that the buffer was an "original" and not a slice. + * + * @return whether or not the buffer is an original + */ + @Deprecated public boolean isRootBuffer() { - return rootBuffer; + return (flags & F_DERIVED) == 0; } @Override @@ -760,4 +969,72 @@ public void close() { release(); } + /** + * Indicates whether this DrillBuf and the supplied one have a "shared fate." + * Having a "shared fate" indicates that the two DrillBufs share a reference + * count, and will both be released at the same time if either of them is + * released. + * @param otherBuf the other buffer to check against + * @return true if the two buffers have a shared fate, false otherwise + */ + public boolean hasSharedFate(final DrillBuf otherBuf) { + return rootRefCnt == otherBuf.rootRefCnt; + } + + private final static int LOG_BYTES_PER_ROW = 10; + /** + * Log this buffer's byte contents in the form of a hex dump. + * + * @param logger where to log to + * @param start the starting byte index + * @param length how many bytes to log + */ + public void logBytes(final Logger logger, final int start, final int length) { + final int roundedStart = (start / LOG_BYTES_PER_ROW) * LOG_BYTES_PER_ROW; + + final StringBuilder sb = new StringBuilder("buffer byte dump\n"); + int index = roundedStart; + for(int nLogged = 0; nLogged < length; nLogged += LOG_BYTES_PER_ROW) { + sb.append(String.format(" [%05d-%05d]", index, index + LOG_BYTES_PER_ROW - 1)); + for(int i = 0; i < LOG_BYTES_PER_ROW; ++i) { + try { + final byte b = getByte(index++); + sb.append(String.format(" 0x%02x", b)); + } catch(IndexOutOfBoundsException ioob) { + sb.append(" "); + } + } + sb.append('\n'); + } + logger.trace(sb.toString()); + } + + /** + * Get the integer id assigned to this DrillBuf for debugging purposes. + * + * @return integer id + */ + public int getId() { + return id; + } + + /** + * Log this buffer's history. + * + * @param logger the logger to use + */ + public void logHistory(final Logger logger) { + if (historicalLog == null) { + logger.warn("DrillBuf[{}] historicalLog not available", id); + } else { + historicalLog.logHistory(logger); + } + } + + public void logHistoryForUdle(final Logger logger, final UnsafeDirectLittleEndian udle) { + final Collection drillBufs = unwrappedGet(udle); + for(final DrillBuf drillBuf : drillBufs) { + drillBuf.logHistory(logger); + } + } } diff --git a/exec/java-exec/src/main/java/io/netty/buffer/FakeAllocator.java b/exec/java-exec/src/main/java/io/netty/buffer/FakeAllocator.java deleted file mode 100644 index 721aff984b0..00000000000 --- a/exec/java-exec/src/main/java/io/netty/buffer/FakeAllocator.java +++ /dev/null @@ -1,164 +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 io.netty.buffer; - -import org.apache.drill.exec.memory.Accountor; -import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.OutOfMemoryException; -import org.apache.drill.exec.ops.FragmentContext; -import org.apache.drill.exec.proto.ExecProtos.FragmentHandle; -import org.apache.drill.exec.util.Pointer; - -class FakeAllocator implements BufferAllocator { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FakeAllocator.class); - - - public static final Accountor FAKE_ACCOUNTOR = new FakeAccountor(); - public static final BufferAllocator FAKE_ALLOCATOR = new FakeAllocator(); - - @Override - public DrillBuf buffer(int size) { - throw new UnsupportedOperationException(); - } - - @Override - public DrillBuf buffer(int minSize, int maxSize) { - throw new UnsupportedOperationException(); - } - - @Override - public ByteBufAllocator getUnderlyingAllocator() { - throw new UnsupportedOperationException(); - } - - @Override - public BufferAllocator getChildAllocator(FragmentContext context, long initialReservation, long maximumReservation, - boolean applyFragmentLimit) - throws OutOfMemoryException { - throw new UnsupportedOperationException(); - } - - @Override - public DrillBuf getEmpty() { - throw new UnsupportedOperationException(); - } - - @Override - public boolean takeOwnership(DrillBuf buf) { - throw new UnsupportedOperationException(); - } - - @Override - public PreAllocator getNewPreAllocator() { - throw new UnsupportedOperationException(); - } - - @Override - public void resetFragmentLimits() { - throw new UnsupportedOperationException(); - } - - @Override - public void setFragmentLimit(long l) { - throw new UnsupportedOperationException(); - } - - @Override - public long getFragmentLimit(){ - throw new UnsupportedOperationException(); - } - - @Override - public void close() { - } - - @Override - public long getAllocatedMemory() { - return 0; - } - - @Override - public long getPeakMemoryAllocation() { - return 0; - } - - static class FakeAccountor extends Accountor { - - public FakeAccountor() { - super(null, false, null, null, 0, 0, true); - } - - @Override - public boolean transferTo(Accountor target, DrillBuf buf, long size) { - throw new UnsupportedOperationException(); - } - - @Override - public long getAvailable() { - throw new UnsupportedOperationException(); - } - - @Override - public long getCapacity() { - throw new UnsupportedOperationException(); - } - - @Override - public long getAllocation() { - throw new UnsupportedOperationException(); - } - - @Override - public boolean reserve(long size) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean forceAdditionalReservation(long size) { - throw new UnsupportedOperationException(); - } - - @Override - public void reserved(long expected, DrillBuf buf) { - throw new UnsupportedOperationException(); - } - - @Override - public void releasePartial(DrillBuf buf, long size) { - - } - - @Override - public void release(DrillBuf buf, long size) { - - } - - @Override - public void close() { - - } - - - } - - @Override - public boolean takeOwnership(DrillBuf buf, Pointer bufOut) { - throw new UnsupportedOperationException(); - } - -} diff --git a/exec/java-exec/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java b/exec/java-exec/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java index 2ca79f09863..0361590efe2 100644 --- a/exec/java-exec/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java +++ b/exec/java-exec/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java @@ -32,9 +32,10 @@ import com.codahale.metrics.Metric; import com.codahale.metrics.MetricFilter; import com.codahale.metrics.MetricRegistry; +import com.google.common.base.Preconditions; -public class PooledByteBufAllocatorL extends PooledByteBufAllocator{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PooledByteBufAllocatorL.class); +public class PooledByteBufAllocatorL extends PooledByteBufAllocator { +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PooledByteBufAllocatorL.class); private static final org.slf4j.Logger memoryLogger = org.slf4j.LoggerFactory.getLogger("drill.allocator"); private static final int MEMORY_LOGGER_FREQUENCY_SECONDS = 60; @@ -56,9 +57,9 @@ public class PooledByteBufAllocatorL extends PooledByteBufAllocator{ private PooledByteBufAllocatorL() { super(true); try { - Field f = PooledByteBufAllocator.class.getDeclaredField("directArenas"); + final Field f = PooledByteBufAllocator.class.getDeclaredField("directArenas"); f.setAccessible(true); - this.directArenas = (PoolArena[]) f.get(this); + directArenas = (PoolArena[]) f.get(this); } catch (Exception e) { throw new RuntimeException("Failure while initializing allocator. Unable to retrieve direct arenas field.", e); } @@ -114,20 +115,19 @@ public boolean matches(String name, Metric metric) { } @Override - protected ByteBuf newHeapBuffer(int initialCapacity, int maxCapacity) { + protected ByteBuf newHeapBuffer(final int initialCapacity, final int maxCapacity) { throw new UnsupportedOperationException("Drill doesn't support using heap buffers."); } @Override - protected UnsafeDirectLittleEndian newDirectBuffer(int initialCapacity, int maxCapacity) { - PoolThreadCache cache = threadCache.get(); - PoolArena directArena = cache.directArena; + protected UnsafeDirectLittleEndian newDirectBuffer(final int initialCapacity, final int maxCapacity) { + final PoolThreadCache cache = threadCache.get(); + final PoolArena directArena = cache.directArena; if (directArena != null) { - if (initialCapacity > directArena.chunkSize) { // This is beyond chunk size so we'll allocate separately. - ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.directBuffer(initialCapacity, maxCapacity); + final ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.directBuffer(initialCapacity, maxCapacity); hugeBufferCount.incrementAndGet(); hugeBufferSize.addAndGet(buf.capacity()); @@ -137,7 +137,7 @@ protected UnsafeDirectLittleEndian newDirectBuffer(int initialCapacity, int maxC } else { // within chunk, use arena. - ByteBuf buf = directArena.allocate(cache, initialCapacity, maxCapacity); + final ByteBuf buf = directArena.allocate(cache, initialCapacity, maxCapacity); if (!(buf instanceof PooledUnsafeDirectByteBuf)) { fail(); } @@ -161,9 +161,8 @@ private UnsupportedOperationException fail() { "Drill requries that the JVM used supports access sun.misc.Unsafe. This platform didn't provide that functionality."); } - @Override - public UnsafeDirectLittleEndian directBuffer(int initialCapacity, int maxCapacity) { + public UnsafeDirectLittleEndian directBuffer(final int initialCapacity, final int maxCapacity) { if (initialCapacity == 0 && maxCapacity == 0) { newDirectBuffer(initialCapacity, maxCapacity); } @@ -172,28 +171,22 @@ public UnsafeDirectLittleEndian directBuffer(int initialCapacity, int maxCapacit } @Override - public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) { + public ByteBuf heapBuffer(final int initialCapacity, final int maxCapacity) { throw new UnsupportedOperationException("Drill doesn't support using heap buffers."); } - - private static void validate(int initialCapacity, int maxCapacity) { - if (initialCapacity < 0) { - throw new IllegalArgumentException("initialCapacity: " + initialCapacity + " (expectd: 0+)"); - } - if (initialCapacity > maxCapacity) { - throw new IllegalArgumentException(String.format( - "initialCapacity: %d (expected: not greater than maxCapacity(%d)", - initialCapacity, maxCapacity)); - } + private static void validate(final int initialCapacity, final int maxCapacity) { + Preconditions.checkArgument(initialCapacity >= 0, + "initialCapacity(%d) was < 0", initialCapacity); + Preconditions.checkArgument(initialCapacity <= maxCapacity, + "initialCapacity(%d) exceeds maxCapacity(%d)", initialCapacity, maxCapacity); } private class MemoryStatusThread extends Thread { - public MemoryStatusThread() { super("memory-status-logger"); - this.setDaemon(true); - this.setName("allocation.logger"); + setDaemon(true); + setName("allocation.logger"); } @Override @@ -205,15 +198,13 @@ public void run() { } catch (InterruptedException e) { return; } - } } - } public void checkAndReset() { if (hugeBufferCount.get() != 0 || normalBufferCount.get() != 0) { - StringBuilder buf = new StringBuilder(); + final StringBuilder buf = new StringBuilder(); buf.append("Large buffers outstanding: "); buf.append(hugeBufferCount.get()); buf.append(" totaling "); @@ -233,8 +224,9 @@ public void checkAndReset() { } } + @Override public String toString() { - StringBuilder buf = new StringBuilder(); + final StringBuilder buf = new StringBuilder(); buf.append(directArenas.length); buf.append(" direct arena(s):"); buf.append(StringUtil.NEWLINE); diff --git a/exec/java-exec/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java b/exec/java-exec/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java index e332b133f35..49c3b7f1544 100644 --- a/exec/java-exec/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java +++ b/exec/java-exec/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java @@ -21,18 +21,83 @@ import io.netty.util.internal.PlatformDependent; import java.nio.ByteOrder; +import java.util.IdentityHashMap; +import java.util.LinkedList; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; +import org.apache.drill.common.StackTrace; import org.apache.drill.exec.util.AssertionUtil; +import org.slf4j.Logger; public final class UnsafeDirectLittleEndian extends WrappedByteBuf { - private static final boolean NATIVE_ORDER = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN; + private static final boolean NATIVE_ORDER = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN; private final AbstractByteBuf wrapped; private final long memoryAddress; + + private static final boolean TRACK_BUFFERS = false; private AtomicLong bufferCount; private AtomicLong bufferSize; private long initCap = -1; + private final static IdentityHashMap bufferMap = new IdentityHashMap<>(); + + @Override + public boolean release() { + return release(1); + } + + @Override + public boolean release(int decrement) { + boolean released = super.release(decrement); + if (TRACK_BUFFERS) { + if (released) { + final Object object; + synchronized (bufferMap) { + object = bufferMap.remove(this); + } + if (object == null) { + throw new IllegalStateException("no such buffer"); + } + + if (initCap != -1) { + bufferCount.decrementAndGet(); + bufferSize.addAndGet(-initCap); + } + } + } + + return released; + } + + + public static int getBufferCount() { + return bufferMap.size(); + } + + public static void releaseBuffers() { + synchronized(bufferMap) { + final Set bufferSet = bufferMap.keySet(); + final LinkedList bufferList = new LinkedList<>(bufferSet); + while(!bufferList.isEmpty()) { + final UnsafeDirectLittleEndian udle = bufferList.removeFirst(); + udle.release(udle.refCnt()); + } + } + } + + public static void logBuffers(final Logger logger) { + synchronized (bufferMap) { + int count = 0; + final Set bufferSet = bufferMap.keySet(); + for (final UnsafeDirectLittleEndian udle : bufferSet) { + final StackTrace stackTrace = bufferMap.get(udle); + ++count; + logger.debug("#" + count + " active buffer allocated at\n" + stackTrace); + } + } + } + UnsafeDirectLittleEndian(LargeBuffer buf) { this(buf, true); } @@ -43,7 +108,7 @@ public final class UnsafeDirectLittleEndian extends WrappedByteBuf { this.bufferSize = bufferSize; // initCap is used if we're tracking memory release. If we're in non-debug mode, we'll skip this. - this.initCap = AssertionUtil.ASSERT_ENABLED ? capacity() : -1; + initCap = AssertionUtil.ASSERT_ENABLED ? capacity() : -1; } private UnsafeDirectLittleEndian(AbstractByteBuf buf, boolean fake) { @@ -52,22 +117,27 @@ private UnsafeDirectLittleEndian(AbstractByteBuf buf, boolean fake) { throw new IllegalStateException("Drill only runs on LittleEndian systems."); } wrapped = buf; - this.memoryAddress = buf.memoryAddress(); + memoryAddress = buf.memoryAddress(); + + if (TRACK_BUFFERS) { + synchronized (bufferMap) { + bufferMap.put(this, new StackTrace()); + } + } } + private long addr(int index) { - return memoryAddress + index; + return memoryAddress + index; } @Override public long getLong(int index) { -// wrapped.checkIndex(index, 8); - long v = PlatformDependent.getLong(addr(index)); - return v; + return PlatformDependent.getLong(addr(index)); } @Override public float getFloat(int index) { - return Float.intBitsToFloat(getInt(index)); + return Float.intBitsToFloat(getInt(index)); } @Override @@ -91,157 +161,137 @@ public ByteBuf order(ByteOrder endianness) { } @Override - public double getDouble(int index) { - return Double.longBitsToDouble(getLong(index)); - } - - @Override - public char getChar(int index) { - return (char) getShort(index); - } + public double getDouble(int index) { + return Double.longBitsToDouble(getLong(index)); + } - @Override - public long getUnsignedInt(int index) { - return getInt(index) & 0xFFFFFFFFL; - } + @Override + public char getChar(int index) { + return (char) getShort(index); + } - @Override - public int getInt(int index) { -// wrapped.checkIndex(index, 4); - int v = PlatformDependent.getInt(addr(index)); - return v; - } + @Override + public long getUnsignedInt(int index) { + return getInt(index) & 0xFFFFFFFFL; + } - @Override - public int getUnsignedShort(int index) { - return getShort(index) & 0xFFFF; - } + @Override + public int getInt(int index) { + return PlatformDependent.getInt(addr(index)); + } - @Override - public short getShort(int index) { -// wrapped.checkIndex(index, 2); - short v = PlatformDependent.getShort(addr(index)); - return v; - } + @Override + public int getUnsignedShort(int index) { + return getShort(index) & 0xFFFF; + } - @Override - public ByteBuf setShort(int index, int value) { - wrapped.checkIndex(index, 2); - _setShort(index, value); - return this; - } + @Override + public short getShort(int index) { + return PlatformDependent.getShort(addr(index)); + } - @Override - public ByteBuf setInt(int index, int value) { - wrapped.checkIndex(index, 4); - _setInt(index, value); - return this; - } + @Override + public ByteBuf setShort(int index, int value) { + wrapped.checkIndex(index, 2); + _setShort(index, value); + return this; + } - @Override - public ByteBuf setLong(int index, long value) { - wrapped.checkIndex(index, 8); - _setLong(index, value); - return this; - } + @Override + public ByteBuf setInt(int index, int value) { + wrapped.checkIndex(index, 4); + _setInt(index, value); + return this; + } - @Override - public ByteBuf setChar(int index, int value) { - setShort(index, value); - return this; - } + @Override + public ByteBuf setLong(int index, long value) { + wrapped.checkIndex(index, 8); + _setLong(index, value); + return this; + } - @Override - public ByteBuf setFloat(int index, float value) { - setInt(index, Float.floatToRawIntBits(value)); - return this; - } + @Override + public ByteBuf setChar(int index, int value) { + setShort(index, value); + return this; + } - @Override - public ByteBuf setDouble(int index, double value) { - setLong(index, Double.doubleToRawLongBits(value)); - return this; - } + @Override + public ByteBuf setFloat(int index, float value) { + setInt(index, Float.floatToRawIntBits(value)); + return this; + } - @Override - public ByteBuf writeShort(int value) { - wrapped.ensureWritable(2); - _setShort(wrapped.writerIndex, value); - wrapped.writerIndex += 2; - return this; - } + @Override + public ByteBuf setDouble(int index, double value) { + setLong(index, Double.doubleToRawLongBits(value)); + return this; + } - @Override - public ByteBuf writeInt(int value) { - wrapped.ensureWritable(4); - _setInt(wrapped.writerIndex, value); - wrapped.writerIndex += 4; - return this; - } - @Override - public ByteBuf writeLong(long value) { - wrapped.ensureWritable(8); - _setLong(wrapped.writerIndex, value); - wrapped.writerIndex += 8; - return this; - } + @Override + public ByteBuf writeShort(int value) { + wrapped.ensureWritable(2); + _setShort(wrapped.writerIndex, value); + wrapped.writerIndex += 2; + return this; + } - @Override - public ByteBuf writeChar(int value) { - writeShort(value); - return this; - } + @Override + public ByteBuf writeInt(int value) { + wrapped.ensureWritable(4); + _setInt(wrapped.writerIndex, value); + wrapped.writerIndex += 4; + return this; + } - @Override - public ByteBuf writeFloat(float value) { - writeInt(Float.floatToRawIntBits(value)); - return this; - } + @Override + public ByteBuf writeLong(long value) { + wrapped.ensureWritable(8); + _setLong(wrapped.writerIndex, value); + wrapped.writerIndex += 8; + return this; + } - @Override - public ByteBuf writeDouble(double value) { - writeLong(Double.doubleToRawLongBits(value)); - return this; - } + @Override + public ByteBuf writeChar(int value) { + writeShort(value); + return this; + } - private void _setShort(int index, int value) { - PlatformDependent.putShort(addr(index), (short) value); - } + @Override + public ByteBuf writeFloat(float value) { + writeInt(Float.floatToRawIntBits(value)); + return this; + } - private void _setInt(int index, int value) { - PlatformDependent.putInt(addr(index), value); - } + @Override + public ByteBuf writeDouble(double value) { + writeLong(Double.doubleToRawLongBits(value)); + return this; + } - private void _setLong(int index, long value) { - PlatformDependent.putLong(addr(index), value); - } + private void _setShort(int index, int value) { + PlatformDependent.putShort(addr(index), (short) value); + } - @Override - public byte getByte(int index) { - return PlatformDependent.getByte(addr(index)); - } + private void _setInt(int index, int value) { + PlatformDependent.putInt(addr(index), value); + } - @Override - public ByteBuf setByte(int index, int value) { - PlatformDependent.putByte(addr(index), (byte) value); - return this; - } + private void _setLong(int index, long value) { + PlatformDependent.putLong(addr(index), value); + } @Override - public boolean release() { - return release(1); + public byte getByte(int index) { + return PlatformDependent.getByte(addr(index)); } @Override - public boolean release(int decrement) { - boolean released = super.release(decrement); - if (released && initCap != -1) { - bufferCount.decrementAndGet(); - bufferSize.addAndGet(-initCap); - } - return released; + public ByteBuf setByte(int index, int value) { + PlatformDependent.putByte(addr(index), (byte) value); + return this; } - - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java index 140e9a87877..a9122e58fdd 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java @@ -76,6 +76,8 @@ public interface ExecConstants { public static final String TEXT_LINE_READER_BUFFER_SIZE = "drill.exec.storage.file.text.buffer.size"; public static final String HAZELCAST_SUBNETS = "drill.exec.cache.hazel.subnets"; public static final String TOP_LEVEL_MAX_ALLOC = "drill.exec.memory.top.max"; + public static final String DEBUG_ALLOCATOR = "drill.exec.memory.debug"; + public static final String ALLOCATION_POLICY = "drill.exec.memory.allocation-policy"; public static final String HTTP_ENABLE = "drill.exec.http.enabled"; public static final String HTTP_PORT = "drill.exec.http.port"; public static final String SYS_STORE_PROVIDER_CLASS = "drill.exec.sys.store.provider.class"; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/TestMemoryRetention.java b/exec/java-exec/src/main/java/org/apache/drill/exec/TestMemoryRetention.java index 37e53894c56..4a738d32289 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/TestMemoryRetention.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/TestMemoryRetention.java @@ -25,12 +25,13 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.drill.common.config.DrillConfig; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.BufferAllocator; +import org.apache.drill.exec.memory.RootAllocator; import com.google.common.collect.Lists; public class TestMemoryRetention { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMemoryRetention.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMemoryRetention.class); static final int SMALL_AVERAGE_BYTES = 1024 * 32; static final int LARGE_BYTES = 32 * 1024 * 1024; @@ -61,9 +62,8 @@ public class TestMemoryRetention { } public static void main(String[] args) throws Exception { - final DrillConfig config = DrillConfig.create(); - final TopLevelAllocator a = new TopLevelAllocator(config); + final BufferAllocator a = new RootAllocator(config); for (int i = 0; i < PARALLEL_THREADS; i++) { Alloc alloc = new Alloc(a); alloc.start(); @@ -71,9 +71,9 @@ public static void main(String[] args) throws Exception { } private static class Alloc extends Thread { - final TopLevelAllocator allocator; + final BufferAllocator allocator; - Alloc(TopLevelAllocator allocator) { + Alloc(BufferAllocator allocator) { this.allocator = allocator; } @@ -107,16 +107,14 @@ public void run() { } else { d.run(); } - } - } private static class Dealloc extends Thread { final List bufs; - final TopLevelAllocator a; + final BufferAllocator a; - public Dealloc(List bufs, TopLevelAllocator a) { + public Dealloc(List bufs, BufferAllocator a) { this.bufs = bufs; this.a = a; } @@ -139,6 +137,4 @@ public void run() { alloc.start(); } } - - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractStreamSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractStreamSerializable.java index ef488d6e113..d5941e0be49 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractStreamSerializable.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractStreamSerializable.java @@ -21,14 +21,12 @@ import java.io.InputStream; import java.io.OutputStream; -public abstract class AbstractStreamSerializable extends LoopedAbstractDrillSerializable{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStreamSerializable.class); +public abstract class AbstractStreamSerializable extends LoopedAbstractDrillSerializable { +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStreamSerializable.class); @Override public abstract void readFromStream(InputStream input) throws IOException; @Override public abstract void writeToStream(OutputStream output) throws IOException; - - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LoopedAbstractDrillSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LoopedAbstractDrillSerializable.java index d2a7458163a..267294b7964 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LoopedAbstractDrillSerializable.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LoopedAbstractDrillSerializable.java @@ -33,12 +33,13 @@ import org.apache.drill.common.util.DataOutputOutputStream; /** - * Helper class that holds the basic functionality to interchangably use the different Drill serializble interfaces. - * This is package private as users should utilize either AbstractDataSerializable or AbstractStreamSerializable instead - * to avoid infinite loops. + * Helper class that holds the basic functionality to interchangeably use + * the different Drill serializble interfaces. This is package private as + * users should utilize either AbstractDataSerializable or AbstractStreamSerializable + * instead to avoid infinite loops. */ abstract class LoopedAbstractDrillSerializable implements DrillSerializable { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LoopedAbstractDrillSerializable.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LoopedAbstractDrillSerializable.class); @Override public void writeExternal(ObjectOutput out) throws IOException { @@ -49,7 +50,6 @@ public void writeExternal(ObjectOutput out) throws IOException { out.write(ba); } - @Override public void read(DataInput input) throws IOException { readFromStream(DataInputInputStream.constructInputStream(input)); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java index a5d2ce8929b..71c904d9a5c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java @@ -52,7 +52,7 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable { private VectorContainer va; private WritableBatch batch; - private BufferAllocator allocator; + private final BufferAllocator allocator; private int recordCount = -1; private BatchSchema.SelectionVectorMode svMode = BatchSchema.SelectionVectorMode.NONE; private SelectionVector2 sv2; @@ -61,7 +61,7 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable { public VectorAccessibleSerializable(BufferAllocator allocator) { this.allocator = allocator; - this.va = new VectorContainer(); + va = new VectorContainer(); } public VectorAccessibleSerializable(WritableBatch batch, BufferAllocator allocator) { @@ -77,16 +77,13 @@ public VectorAccessibleSerializable(WritableBatch batch, BufferAllocator allocat */ public VectorAccessibleSerializable(WritableBatch batch, SelectionVector2 sv2, BufferAllocator allocator) { this.allocator = allocator; - if (batch != null) { - this.batch = batch; - } + this.batch = batch; if (sv2 != null) { this.sv2 = sv2; - this.svMode = BatchSchema.SelectionVectorMode.TWO_BYTE; + svMode = BatchSchema.SelectionVectorMode.TWO_BYTE; } } - /** * Reads from an InputStream and parses a RecordBatchDef. From this, we construct a SelectionVector2 if it exits * and construct the vectors and add them to a vector container @@ -95,8 +92,8 @@ public VectorAccessibleSerializable(WritableBatch batch, SelectionVector2 sv2, B */ @Override public void readFromStream(InputStream input) throws IOException { - VectorContainer container = new VectorContainer(); - UserBitShared.RecordBatchDef batchDef = UserBitShared.RecordBatchDef.parseDelimitedFrom(input); + final VectorContainer container = new VectorContainer(); + final UserBitShared.RecordBatchDef batchDef = UserBitShared.RecordBatchDef.parseDelimitedFrom(input); recordCount = batchDef.getRecordCount(); if (batchDef.hasCarriesTwoByteSelectionVector() && batchDef.getCarriesTwoByteSelectionVector()) { @@ -107,12 +104,12 @@ public void readFromStream(InputStream input) throws IOException { sv2.getBuffer().setBytes(0, input, recordCount * SelectionVector2.RECORD_SIZE); svMode = BatchSchema.SelectionVectorMode.TWO_BYTE; } - List vectorList = Lists.newArrayList(); - List fieldList = batchDef.getFieldList(); + final List vectorList = Lists.newArrayList(); + final List fieldList = batchDef.getFieldList(); for (SerializedField metaData : fieldList) { - int dataLength = metaData.getBufferLength(); - MaterializedField field = MaterializedField.create(metaData); - DrillBuf buf = allocator.buffer(dataLength); + final int dataLength = metaData.getBufferLength(); + final MaterializedField field = MaterializedField.create(metaData); + final DrillBuf buf = allocator.buffer(dataLength); final ValueVector vector; try { buf.writeBytes(input, dataLength); @@ -129,7 +126,6 @@ public void readFromStream(InputStream input) throws IOException { va = container; } - public void writeToStreamAndRetain(OutputStream output) throws IOException { retain = true; writeToStream(output); @@ -145,8 +141,8 @@ public void writeToStream(OutputStream output) throws IOException { Preconditions.checkNotNull(output); final Timer.Context timerContext = metrics.timer(WRITER_TIMER).time(); - DrillBuf[] incomingBuffers = batch.getBuffers(); - UserBitShared.RecordBatchDef batchDef = batch.getDef(); + final DrillBuf[] incomingBuffers = batch.getBuffers(); + final UserBitShared.RecordBatchDef batchDef = batch.getDef(); /* DrillBuf associated with the selection vector */ DrillBuf svBuf = null; @@ -202,5 +198,4 @@ public VectorContainer get() { public SelectionVector2 getSv2() { return sv2; } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java index c642c4a7d23..2fec6b62a0e 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java @@ -31,13 +31,15 @@ import java.util.Properties; import java.util.Vector; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.exceptions.UserException; import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.coord.ClusterCoordinator; import org.apache.drill.exec.coord.zk.ZKClusterCoordinator; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.OutOfMemoryException; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.proto.GeneralRPCProtos.Ack; import org.apache.drill.exec.proto.UserBitShared; @@ -71,7 +73,7 @@ public class DrillClient implements Closeable, ConnectionThrottle { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClient.class); - DrillConfig config; + private final DrillConfig config; private UserClient client; private UserProperties props = null; private volatile ClusterCoordinator clusterCoordinator; @@ -85,50 +87,55 @@ public class DrillClient implements Closeable, ConnectionThrottle { private final boolean isDirectConnection; // true if the connection bypasses zookeeper and connects directly to a drillbit private EventLoopGroup eventLoopGroup; - public DrillClient() { + public DrillClient() throws OutOfMemoryException { this(DrillConfig.create(), false); } - public DrillClient(boolean isDirect) { + public DrillClient(boolean isDirect) throws OutOfMemoryException { this(DrillConfig.create(), isDirect); } - public DrillClient(String fileName) { + public DrillClient(String fileName) throws OutOfMemoryException { this(DrillConfig.create(fileName), false); } - public DrillClient(DrillConfig config) { + public DrillClient(DrillConfig config) throws OutOfMemoryException { this(config, null, false); } - public DrillClient(DrillConfig config, boolean isDirect) { + public DrillClient(DrillConfig config, boolean isDirect) + throws OutOfMemoryException { this(config, null, isDirect); } - public DrillClient(DrillConfig config, ClusterCoordinator coordinator) { + public DrillClient(DrillConfig config, ClusterCoordinator coordinator) + throws OutOfMemoryException { this(config, coordinator, null, false); } - public DrillClient(DrillConfig config, ClusterCoordinator coordinator, boolean isDirect) { + public DrillClient(DrillConfig config, ClusterCoordinator coordinator, boolean isDirect) + throws OutOfMemoryException { this(config, coordinator, null, isDirect); } - public DrillClient(DrillConfig config, ClusterCoordinator coordinator, BufferAllocator allocator) { + public DrillClient(DrillConfig config, ClusterCoordinator coordinator, BufferAllocator allocator) + throws OutOfMemoryException { this(config, coordinator, allocator, false); } public DrillClient(DrillConfig config, ClusterCoordinator coordinator, BufferAllocator allocator, boolean isDirect) { // if isDirect is true, the client will connect directly to the drillbit instead of // going thru the zookeeper - this.isDirectConnection = isDirect; - this.ownsZkConnection = coordinator == null && !isDirect; - this.ownsAllocator = allocator == null; - this.allocator = ownsAllocator ? new TopLevelAllocator(config) : allocator; + ownsAllocator = allocator == null; + this.allocator = ownsAllocator ? new RootAllocator(config) : allocator; this.config = config; - this.clusterCoordinator = coordinator; - this.reconnectTimes = config.getInt(ExecConstants.BIT_RETRY_TIMES); - this.reconnectDelay = config.getInt(ExecConstants.BIT_RETRY_DELAY); - this.supportComplexTypes = config.getBoolean(ExecConstants.CLIENT_SUPPORT_COMPLEX_TYPES); + + clusterCoordinator = coordinator; + isDirectConnection = isDirect; + ownsZkConnection = coordinator == null; + reconnectTimes = config.getInt(ExecConstants.BIT_RETRY_TIMES); + reconnectDelay = config.getInt(ExecConstants.BIT_RETRY_DELAY); + supportComplexTypes = config.getBoolean(ExecConstants.CLIENT_SUPPORT_COMPLEX_TYPES); } public DrillConfig getConfig() { @@ -258,13 +265,16 @@ public void close() { this.client.close(); } if (this.ownsAllocator && allocator != null) { - allocator.close(); + DrillAutoCloseables.closeNoChecked(allocator); } if (ownsZkConnection) { - try { - this.clusterCoordinator.close(); - } catch (IOException e) { - logger.warn("Error while closing Cluster Coordinator.", e); + if (clusterCoordinator != null) { + try { + clusterCoordinator.close(); + clusterCoordinator = null; + } catch (IOException e) { + logger.warn("Error while closing Cluster Coordinator.", e); + } } } if (eventLoopGroup != null) { @@ -371,6 +381,10 @@ public void queryCompleted(QueryState state) { private void fail(Exception ex) { logger.debug("Submission failed.", ex); + final Throwable cause = ex.getCause(); + if (cause != null) { + logger.debug("Submission failure cause.", ex.getCause()); + } future.setException(ex); future.set(results); } @@ -385,6 +399,14 @@ public List getResults() throws RpcException{ try { return future.get(); } catch (Throwable t) { + /* + * Since we're not going to return the result to the caller + * to clean up, we have to do it. + */ + for(final QueryDataBatch queryDataBatch : results) { + queryDataBatch.release(); + } + throw RpcException.mapException(t); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java index 55d9cf3ee94..1431d3c1609 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java @@ -21,9 +21,10 @@ import java.io.FileInputStream; import java.util.List; +import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.cache.VectorAccessibleSerializable; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.record.BatchSchema; import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode; import org.apache.drill.exec.record.MaterializedField; @@ -39,8 +40,12 @@ import com.google.common.collect.Lists; public class DumpCat { + private final DrillConfig drillConfig = DrillConfig.create(); + private final BufferAllocator allocator; - private final static BufferAllocator allocator = new TopLevelAllocator(); + public DumpCat() throws Exception { + allocator = new RootAllocator(drillConfig); + } public static void main(String args[]) throws Exception { DumpCat dumpCat = new DumpCat(); @@ -174,7 +179,7 @@ protected void doQuery(FileInputStream input) throws Exception{ BatchMetaInfo aggBatchMetaInfo = new BatchMetaInfo(); while (input.available() > 0) { - VectorAccessibleSerializable vcSerializable = new VectorAccessibleSerializable(DumpCat.allocator); + VectorAccessibleSerializable vcSerializable = new VectorAccessibleSerializable(allocator); vcSerializable.readFromStream(input); VectorContainer vectorContainer = (VectorContainer) vcSerializable.get(); @@ -221,7 +226,7 @@ protected void doBatch(FileInputStream input, int targetBatchNum, boolean showHe VectorAccessibleSerializable vcSerializable = null; while (input.available() > 0 && batchNum ++ < targetBatchNum) { - vcSerializable = new VectorAccessibleSerializable(DumpCat.allocator); + vcSerializable = new VectorAccessibleSerializable(allocator); vcSerializable.readFromStream(input); if (batchNum != targetBatchNum) { @@ -243,7 +248,6 @@ protected void doBatch(FileInputStream input, int targetBatchNum, boolean showHe } } - private void showSingleBatch (VectorAccessibleSerializable vcSerializable, boolean showHeader) { VectorContainer vectorContainer = (VectorContainer)vcSerializable.get(); @@ -252,8 +256,8 @@ private void showSingleBatch (VectorAccessibleSerializable vcSerializable, boole System.out.println(getBatchMetaInfo(vcSerializable).toString()); System.out.println("Schema Information"); - for (VectorWrapper w : vectorContainer) { - MaterializedField field = w.getValueVector().getField(); + for (final VectorWrapper w : vectorContainer) { + final MaterializedField field = w.getValueVector().getField(); System.out.println (String.format("name : %s, minor_type : %s, data_mode : %s", field.toExpr(), field.getType().getMinorType().toString(), @@ -266,12 +270,11 @@ private void showSingleBatch (VectorAccessibleSerializable vcSerializable, boole VectorUtil.showVectorAccessibleContent(vectorContainer); } - /* Get batch meta info : rows, selectedRows, dataSize */ private BatchMetaInfo getBatchMetaInfo(VectorAccessibleSerializable vcSerializable) { VectorAccessible vectorContainer = vcSerializable.get(); - int rows =0; + int rows = 0; int selectedRows = 0; int totalDataSize = 0; @@ -282,11 +285,10 @@ private BatchMetaInfo getBatchMetaInfo(VectorAccessibleSerializable vcSerializab selectedRows = vcSerializable.getSv2().getCount(); } - for (VectorWrapper w : vectorContainer) { + for (final VectorWrapper w : vectorContainer) { totalDataSize += w.getValueVector().getBufferSize(); } return new BatchMetaInfo(rows, selectedRows, totalDataSize); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java index 8cb4a0b579e..16873b62da0 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java @@ -23,14 +23,15 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.exceptions.UserException; import org.apache.drill.exec.client.QuerySubmitter.Format; import org.apache.drill.exec.exception.SchemaChangeException; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; -import org.apache.drill.exec.proto.UserBitShared.QueryData; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.proto.UserBitShared.QueryId; +import org.apache.drill.exec.proto.UserBitShared.QueryData; import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState; import org.apache.drill.exec.record.RecordBatchLoader; import org.apache.drill.exec.rpc.user.ConnectionThrottle; @@ -42,19 +43,18 @@ public class PrintingResultsListener implements UserResultsListener { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PrintingResultsListener.class); - - AtomicInteger count = new AtomicInteger(); - private CountDownLatch latch = new CountDownLatch(1); - RecordBatchLoader loader; - Format format; - int columnWidth; - BufferAllocator allocator; - volatile UserException exception; - QueryId queryId; - Stopwatch w = new Stopwatch(); + private final AtomicInteger count = new AtomicInteger(); + private final CountDownLatch latch = new CountDownLatch(1); + private RecordBatchLoader loader; + private Format format; + private final int columnWidth; + private final BufferAllocator allocator; + private volatile UserException exception; + private QueryId queryId; + private final Stopwatch w = new Stopwatch(); public PrintingResultsListener(DrillConfig config, Format format, int columnWidth) { - this.allocator = new TopLevelAllocator(config); + allocator = new RootAllocator(config); // TODO(DRILL-3335) shouldn't this be a child of the DrillClient? loader = new RecordBatchLoader(allocator); this.format = format; this.columnWidth = columnWidth; @@ -70,7 +70,7 @@ public void submissionFailed(UserException ex) { @Override public void queryCompleted(QueryState state) { - allocator.close(); + DrillAutoCloseables.closeNoChecked(allocator); latch.countDown(); System.out.println("Total rows returned : " + count.get() + ". Returned in " + w.elapsed(TimeUnit.MILLISECONDS) + "ms."); @@ -125,5 +125,4 @@ public void queryIdArrived(QueryId queryId) { w.start(); this.queryId = queryId; } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java index c2768467ab1..d5fed782eeb 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java @@ -19,14 +19,15 @@ import org.apache.drill.exec.work.foreman.ForemanException; -public class FragmentSetupException extends ForemanException{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentSetupException.class); +public class FragmentSetupException extends ForemanException { + private static final long serialVersionUID = -7593886431264516650L; +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentSetupException.class); public FragmentSetupException() { - super(); } - public FragmentSetupException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + public FragmentSetupException(String message, Throwable cause, boolean enableSuppression, + boolean writableStackTrace) { super(message, cause, enableSuppression, writableStackTrace); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java index e27234fdb22..7da9eaff794 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java @@ -21,12 +21,9 @@ import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.drill.common.types.TypeProtos.DataMode; -//import org.apache.drill.common.types.DataMode; -import org.apache.drill.common.types.MinorType; import org.apache.drill.common.types.TypeProtos; import org.apache.drill.exec.expr.holders.VarCharHolder; import org.apache.drill.exec.vector.complex.MapUtility; -import org.apache.drill.exec.vector.complex.impl.SingleMapReaderImpl; import org.apache.drill.exec.vector.complex.reader.FieldReader; import org.apache.drill.exec.vector.complex.writer.BaseWriter; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java index 49f581fcd09..db3b6b4db5b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java @@ -690,11 +690,15 @@ public static class Lpad implements DrillSimpleFunc{ @Output VarCharHolder out; + @Override public void setup() { } + @Override public void eval() { - buffer = buffer.reallocIfNeeded((int) length.value*2); + final long theLength = length.value; + final int lengthNeeded = (int) (theLength <= 0 ? 0 : theLength * 2); + buffer = buffer.reallocIfNeeded(lengthNeeded); byte currentByte = 0; int id = 0; //get the char length of text. @@ -703,29 +707,29 @@ public void eval() { //get the char length of fill. int fillCharCount = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharLength(fill.buffer, fill.start, fill.end); - if (length.value <= 0) { + if (theLength <= 0) { //case 1: target length is <=0, then return an empty string. out.buffer = buffer; out.start = out.end = 0; - } else if (length.value == textCharCount || (length.value > textCharCount && fillCharCount == 0) ) { + } else if (theLength == textCharCount || (theLength > textCharCount && fillCharCount == 0) ) { //case 2: target length is same as text's length, or need fill into text but "fill" is empty, then return text directly. out.buffer = text.buffer; out.start = text.start; out.end = text.end; - } else if (length.value < textCharCount) { + } else if (theLength < textCharCount) { //case 3: truncate text on the right side. It's same as substring(text, 1, length). out.buffer = text.buffer; out.start = text.start; - out.end = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharPosition(text.buffer, text.start, text.end, (int)length.value); - } else if (length.value > textCharCount) { - //case 4: copy "fill" on left. Total # of char to copy : length.value - textCharCount + out.end = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharPosition(text.buffer, text.start, text.end, (int) theLength); + } else if (theLength > textCharCount) { + //case 4: copy "fill" on left. Total # of char to copy : theLength - textCharCount int count = 0; out.buffer = buffer; out.start = out.end = 0; - while (count < length.value - textCharCount) { + while (count < theLength - textCharCount) { for (id = fill.start; id < fill.end; id++) { - if (count == length.value - textCharCount) { + if (count == theLength - textCharCount) { break; } @@ -763,11 +767,15 @@ public static class Rpad implements DrillSimpleFunc{ @Output VarCharHolder out; + @Override public void setup() { } + @Override public void eval() { - buffer = buffer.reallocIfNeeded((int) length.value*2); + final long theLength = length.value; + final int lengthNeeded = (int) (theLength <= 0 ? 0 : theLength * 2); + buffer = buffer.reallocIfNeeded(lengthNeeded); byte currentByte = 0; int id = 0; @@ -777,21 +785,21 @@ public void eval() { //get the char length of fill. int fillCharCount = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharLength(fill.buffer, fill.start, fill.end); - if (length.value <= 0) { + if (theLength <= 0) { //case 1: target length is <=0, then return an empty string. out.buffer = buffer; out.start = out.end = 0; - } else if (length.value == textCharCount || (length.value > textCharCount && fillCharCount == 0) ) { + } else if (theLength == textCharCount || (theLength > textCharCount && fillCharCount == 0) ) { //case 2: target length is same as text's length, or need fill into text but "fill" is empty, then return text directly. out.buffer = text.buffer; out.start = text.start; out.end = text.end; - } else if (length.value < textCharCount) { + } else if (theLength < textCharCount) { //case 3: truncate text on the right side. It's same as substring(text, 1, length). out.buffer = text.buffer; out.start = text.start; - out.end = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharPosition(text.buffer, text.start, text.end, (int)length.value); - } else if (length.value > textCharCount) { + out.end = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.getUTF8CharPosition(text.buffer, text.start, text.end, (int) theLength); + } else if (theLength > textCharCount) { //case 4: copy "text" into "out", then copy "fill" on the right. out.buffer = buffer; out.start = out.end = 0; @@ -800,12 +808,12 @@ public void eval() { out.buffer.setByte(out.end++, text.buffer.getByte(id)); } - //copy "fill" on right. Total # of char to copy : length.value - textCharCount + //copy "fill" on right. Total # of char to copy : theLength - textCharCount int count = 0; - while (count < length.value - textCharCount) { + while (count < theLength - textCharCount) { for (id = fill.start; id < fill.end; id++) { - if (count == length.value - textCharCount) { + if (count == theLength - textCharCount) { break; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java deleted file mode 100644 index ad6a7877b72..00000000000 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java +++ /dev/null @@ -1,462 +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.memory; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.DrillBuf; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ConcurrentMap; - -import org.apache.drill.common.config.DrillConfig; -import org.apache.drill.exec.ExecConstants; -import org.apache.drill.exec.ops.FragmentContext; -import org.apache.drill.exec.proto.ExecProtos.FragmentHandle; -import org.apache.drill.exec.proto.helper.QueryIdHelper; -import org.apache.drill.exec.util.AssertionUtil; - -import com.google.common.collect.LinkedListMultimap; -import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; - -public class Accountor { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Accountor.class); - - private static final boolean ENABLE_ACCOUNTING = AssertionUtil.isAssertionsEnabled(); - private final AtomicRemainder remainder; - private final long total; - private ConcurrentMap buffers = Maps.newConcurrentMap(); - private final FragmentHandle handle; - private String fragmentStr; - private Accountor parent; - - private final boolean errorOnLeak; - // some operators are no subject to the fragment limit. They set the applyFragmentLimit to false - - private final boolean enableFragmentLimit; - private final double fragmentMemOvercommitFactor; - - private final boolean DEFAULT_ENABLE_FRAGMENT_LIMIT=false; - private final double DEFAULT_FRAGMENT_MEM_OVERCOMMIT_FACTOR=1.5; - - private final boolean applyFragmentLimit; - - private final FragmentContext fragmentContext; - long fragmentLimit; - - private long peakMemoryAllocation = 0; - - // The top level Allocator has an accountor that keeps track of all the FragmentContexts currently executing. - // This enables the top level accountor to calculate a new fragment limit whenever necessary. - private final List fragmentContexts; - - public Accountor(DrillConfig config, boolean errorOnLeak, FragmentContext context, Accountor parent, long max, long preAllocated, boolean applyFragLimit) { - // TODO: fix preallocation stuff - this.errorOnLeak = errorOnLeak; - AtomicRemainder parentRemainder = parent != null ? parent.remainder : null; - this.parent = parent; - - boolean enableFragmentLimit; - double fragmentMemOvercommitFactor; - - try { - enableFragmentLimit = config.getBoolean(ExecConstants.ENABLE_FRAGMENT_MEMORY_LIMIT); - fragmentMemOvercommitFactor = config.getDouble(ExecConstants.FRAGMENT_MEM_OVERCOMMIT_FACTOR); - }catch(Exception e){ - enableFragmentLimit = DEFAULT_ENABLE_FRAGMENT_LIMIT; - fragmentMemOvercommitFactor = DEFAULT_FRAGMENT_MEM_OVERCOMMIT_FACTOR; - } - this.enableFragmentLimit = enableFragmentLimit; - this.fragmentMemOvercommitFactor = fragmentMemOvercommitFactor; - - - this.applyFragmentLimit=applyFragLimit; - - this.remainder = new AtomicRemainder(errorOnLeak, parentRemainder, max, preAllocated, applyFragmentLimit); - this.total = max; - this.fragmentContext=context; - this.handle = (context!=null) ? context.getHandle() : null; - this.fragmentStr= (handle!=null) ? ( handle.getMajorFragmentId()+":"+handle.getMinorFragmentId() ) : "0:0"; - this.fragmentLimit=this.total; // Allow as much as possible to start with; - if (ENABLE_ACCOUNTING) { - buffers = Maps.newConcurrentMap(); - } else { - buffers = null; - } - this.fragmentContexts = new ArrayList(); - if(parent!=null && parent.parent==null){ // Only add the fragment context to the fragment level accountor - synchronized(this) { - addFragmentContext(this.fragmentContext); - } - } - } - - public boolean transferTo(Accountor target, DrillBuf buf, long size) { - return transfer(target, buf, size, true); - } - - public boolean transferIn(DrillBuf buf, long size) { - return transfer(this, buf, size, false); - } - - private boolean transfer(Accountor target, DrillBuf buf, long size, boolean release) { - boolean withinLimit = target.forceAdditionalReservation(size); - if(release){ - release(buf, size); - } - - if (ENABLE_ACCOUNTING) { - target.buffers.put(buf, new DebugStackTrace(buf.capacity(), Thread.currentThread().getStackTrace())); - } - return withinLimit; - } - - public long getAvailable() { - if (parent != null) { - return Math.min(parent.getAvailable(), getCapacity() - getAllocation()); - } - return getCapacity() - getAllocation(); - } - - public long getCapacity() { - return fragmentLimit; - } - - public long getAllocation() { - return remainder.getUsed(); - } - - public long getPeakMemoryAllocation() { - return peakMemoryAllocation; - } - - public boolean reserve(long size) { - logger.trace("Fragment:"+fragmentStr+" Reserved "+size+" bytes. Total Allocated: "+getAllocation()); - boolean status = remainder.get(size, this.applyFragmentLimit); - peakMemoryAllocation = Math.max(peakMemoryAllocation, getAllocation()); - return status; - } - - public boolean forceAdditionalReservation(long size) { - if (size > 0) { - boolean status = remainder.forceGet(size); - peakMemoryAllocation = Math.max(peakMemoryAllocation, getAllocation()); - return status; - } else { - return true; - } - } - - public void reserved(long expected, DrillBuf buf) { - // make sure to take away the additional memory that happened due to rounding. - - long additional = buf.capacity() - expected; - if (additional > 0) { - remainder.forceGet(additional); - } - - if (ENABLE_ACCOUNTING) { - buffers.put(buf, new DebugStackTrace(buf.capacity(), Thread.currentThread().getStackTrace())); - } - - peakMemoryAllocation = Math.max(peakMemoryAllocation, getAllocation()); - } - - - public void releasePartial(DrillBuf buf, long size) { - remainder.returnAllocation(size); - if (ENABLE_ACCOUNTING) { - if (buf != null) { - DebugStackTrace dst = buffers.get(buf); - if (dst == null) { - throw new IllegalStateException("Partially releasing a buffer that has already been released. Buffer: " + buf); - } - dst.size -= size; - if (dst.size < 0) { - throw new IllegalStateException("Partially releasing a buffer that has already been released. Buffer: " + buf); - } - } - } - } - - void release(long size) { - remainder.returnAllocation(size); - } - - public void release(DrillBuf buf, long size) { - remainder.returnAllocation(size); - if (ENABLE_ACCOUNTING) { - if (buf != null && buffers.remove(buf) == null) { - throw new IllegalStateException("Releasing a buffer that has already been released. Buffer: " + buf); - } - } - } - - private void addFragmentContext(FragmentContext c) { - if (parent != null){ - parent.addFragmentContext(c); - }else { - if(logger.isTraceEnabled()) { - FragmentHandle hndle; - String fragStr; - if(c!=null) { - hndle = c.getHandle(); - fragStr = (hndle != null) ? (hndle.getMajorFragmentId() + ":" + hndle.getMinorFragmentId()) : "[Null Fragment Handle]"; - }else{ - fragStr = "[Null Context]"; - } - fragStr+=" (Object Id: "+System.identityHashCode(c)+")"; - StackTraceElement[] ste = (new Throwable()).getStackTrace(); - StringBuffer sb = new StringBuffer(); - for (StackTraceElement s : ste) { - sb.append(s.toString()); - sb.append("\n"); - } - - logger.trace("Fragment " + fragStr + " added to root accountor.\n"+sb.toString()); - } - synchronized(this) { - fragmentContexts.add(c); - } - } - } - - private void removeFragmentContext(FragmentContext c) { - if (parent != null){ - if (parent.parent==null){ - // only fragment level allocators will have the fragment context saved - parent.removeFragmentContext(c); - } - }else{ - if(logger.isDebugEnabled()) { - FragmentHandle hndle; - String fragStr; - if (c != null) { - hndle = c.getHandle(); - fragStr = (hndle != null) ? (hndle.getMajorFragmentId() + ":" + hndle.getMinorFragmentId()) : "[Null Fragment Handle]"; - } else { - fragStr = "[Null Context]"; - } - fragStr += " (Object Id: " + System.identityHashCode(c) + ")"; - logger.trace("Fragment " + fragStr + " removed from root accountor"); - } - synchronized(this) { - fragmentContexts.remove(c); - } - } - } - - public long resetFragmentLimits(){ - // returns the new capacity - if(!this.enableFragmentLimit){ - return getCapacity(); - } - - if(parent!=null){ - parent.resetFragmentLimits(); - }else { - //Get remaining memory available per fragment and distribute it EQUALLY among all the fragments. - //Fragments get the memory limit added to the amount already allocated. - //This favours fragments that are already running which will get a limit greater than newly started fragments. - //If the already running fragments end quickly, their limits will be assigned back to the remaining fragments - //quickly. If they are long running, then we want to favour them with larger limits anyway. - synchronized (this) { - int nFragments=fragmentContexts.size(); - long allocatedMemory=0; - for(FragmentContext fragment: fragmentContexts){ - BufferAllocator a = fragment.getAllocator(); - if(a!=null) { - allocatedMemory += fragment.getAllocator().getAllocatedMemory(); - } - } - if(logger.isTraceEnabled()) { - logger.trace("Resetting Fragment Memory Limit: total Available memory== "+total - +" Total Allocated Memory :"+allocatedMemory - +" Number of fragments: "+nFragments - + " fragmentMemOvercommitFactor: "+fragmentMemOvercommitFactor - + " Root fragment limit: "+this.fragmentLimit + "(Root obj: "+System.identityHashCode(this)+")" - ); - } - if(nFragments>0) { - long rem = (total - allocatedMemory) / nFragments; - for (FragmentContext fragment : fragmentContexts) { - fragment.setFragmentLimit((long) (rem * fragmentMemOvercommitFactor)); - } - } - if(logger.isTraceEnabled() && false){ - StringBuffer sb= new StringBuffer(); - sb.append("[root](0:0)"); - sb.append("Allocated memory: "); - sb.append(this.getAllocation()); - sb.append(" Fragment Limit: "); - sb.append(this.getFragmentLimit()); - logger.trace(sb.toString()); - for(FragmentContext fragment: fragmentContexts){ - sb= new StringBuffer(); - if (handle != null) { - sb.append("["); - sb.append(QueryIdHelper.getQueryId(handle.getQueryId())); - sb.append("]("); - sb.append(handle.getMajorFragmentId()); - sb.append(":"); - sb.append(handle.getMinorFragmentId()); - sb.append(")"); - }else{ - sb.append("[fragment](0:0)"); - } - sb.append("Allocated memory: "); - sb.append(fragment.getAllocator().getAllocatedMemory()); - sb.append(" Fragment Limit: "); - sb.append(fragment.getAllocator().getFragmentLimit()); - logger.trace(sb.toString()); - } - logger.trace("Resetting Complete"); - } - } - } - return getCapacity(); - } - - public void close() { - // remove the fragment context and reset fragment limits whenever an allocator closes - if(parent!=null && parent.parent==null) { - logger.debug("Fragment " + fragmentStr + " accountor being closed"); - removeFragmentContext(fragmentContext); - } - resetFragmentLimits(); - - if (ENABLE_ACCOUNTING && !buffers.isEmpty()) { - StringBuffer sb = new StringBuffer(); - sb.append("Attempted to close accountor with "); - sb.append(buffers.size()); - sb.append(" buffer(s) still allocated"); - if (handle != null) { - sb.append("for QueryId: "); - sb.append(QueryIdHelper.getQueryId(handle.getQueryId())); - sb.append(", MajorFragmentId: "); - sb.append(handle.getMajorFragmentId()); - sb.append(", MinorFragmentId: "); - sb.append(handle.getMinorFragmentId()); - } - sb.append(".\n"); - - Multimap multi = LinkedListMultimap.create(); - for (DebugStackTrace t : buffers.values()) { - multi.put(t, t); - } - - for (DebugStackTrace entry : multi.keySet()) { - Collection allocs = multi.get(entry); - - sb.append("\n\n\tTotal "); - sb.append(allocs.size()); - sb.append(" allocation(s) of byte size(s): "); - for (DebugStackTrace alloc : allocs) { - sb.append(alloc.size); - sb.append(", "); - } - - sb.append("at stack location:\n"); - entry.addToString(sb); - } - if (!buffers.isEmpty()) { - IllegalStateException e = new IllegalStateException(sb.toString()); - if (errorOnLeak) { - throw e; - } else { - logger.warn("Memory leaked.", e); - } - } - } - - remainder.close(); - - } - - public void setFragmentLimit(long add) { - // We ADD the limit to the current allocation. If none has been allocated, this - // sets a new limit. If memory has already been allocated, the fragment gets its - // limit based on the allocation, though this might still result in reducing the - // limit. - - if (parent != null && parent.parent==null) { // This is a fragment level accountor - this.fragmentLimit=getAllocation()+add; - this.remainder.setLimit(this.fragmentLimit); - logger.trace("Fragment "+fragmentStr+" memory limit set to "+this.fragmentLimit); - } - } - - public long getFragmentLimit(){ - return this.fragmentLimit; - } - - public class DebugStackTrace { - - private StackTraceElement[] elements; - private long size; - - public DebugStackTrace(long size, StackTraceElement[] elements) { - super(); - this.elements = elements; - this.size = size; - } - - public void addToString(StringBuffer sb) { - for (int i = 3; i < elements.length; i++) { - sb.append("\t\t"); - sb.append(elements[i]); - sb.append("\n"); - } - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + Arrays.hashCode(elements); -// result = prime * result + (int) (size ^ (size >>> 32)); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - DebugStackTrace other = (DebugStackTrace) obj; - if (!Arrays.equals(elements, other.elements)) { - return false; - } - // weird equal where size doesn't matter for multimap purposes. -// if (size != other.size) -// return false; - return true; - } - - } - -} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java new file mode 100644 index 00000000000..4f1a1bd5c0f --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java @@ -0,0 +1,38 @@ +/** + * 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.memory; + +/** + * Implicitly specifies an allocation policy by providing a factory method to + * create an enforcement agent. + * + *

Allocation policies are meant to be global, and may not work properly if + * different allocators are given different policies. These are designed to + * be supplied to the root-most allocator only, and then shared with descendant + * (child) allocators.

+ */ +public interface AllocationPolicy { + /** + * Create an allocation policy enforcement agent. Each newly created allocator should + * call this in order to obtain its own agent. + * + * @return the newly instantiated agent; if an agent's implementation is stateless, + * this may return a sharable singleton + */ + AllocationPolicyAgent newAgent(); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java new file mode 100644 index 00000000000..ad51ee6a135 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java @@ -0,0 +1,69 @@ +/** + * 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.memory; + +/** + * Per-allocator enforcement agent for allocation policies; created by + * {@link AllocationPolicy#newAgent()}. + */ +public interface AllocationPolicyAgent extends AutoCloseable { + /** + * Checks to see if creating a new allocator using the given specifications + * is allowed; should throw an exception if not. + * + * @param parentAllocator the parent allocator + * @param initReservation initial reservation the allocator should have + * @param maxAllocation the maximum allocation the allocator will allow + * @param flags the allocation option flags + * @throws OutOfMemoryException if the new allocator shouldn't be created + */ + void checkNewAllocator(BufferAllocator parentAllocator, + long initReservation, long maxAllocation, int flags); + + /** + * Get the currently applicable memory limit for the provided allocator. + * The interpretation of this value varies with the allocation policy in + * use, and each policy should describe what to expect. + * + * @param bufferAllocator the allocator + * @return the memory limit + */ + long getMemoryLimit(BufferAllocator bufferAllocator); + + /** + * Initialize the agent for a newly created allocator. Should be called from + * the allocator's constructor to initialize the agent for the allocator. + * + * @param bufferAllocator the newly created allocator. + */ + void initializeAllocator(BufferAllocator bufferAllocator); + + /** + * Indicate if any available memory owned by this allocator should + * be released to its parent. Allocators may use this to limit the + * amount of unused memory they retain for future requests; agents may + * request that memory be returned if there is currently a high demand + * for memory that other allocators could use if this allocator + * doesn't need it. + * + * @param bufferAllocator + * @return true if available memory owned by this allocator should be given + * back to its parent + */ + boolean shouldReleaseToParent(BufferAllocator bufferAllocator); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java new file mode 100644 index 00000000000..18035724185 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java @@ -0,0 +1,152 @@ +/** + * 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.memory; + +import com.google.common.base.Preconditions; + +import io.netty.buffer.DrillBuf; + +/** + * Supports cumulative allocation reservation. Clients may increase the size of + * the reservation repeatedly until they call for an allocation of the current + * total size. The reservation can only be used once, and will throw an exception + * if it is used more than once. + * + *

For the purposes of airtight memory accounting, the reservation must be close()d + * whether it is used or not. + */ +public abstract class AllocationReservation implements AutoCloseable { + private int nBytes = 0; + private boolean used = false; + private boolean closed = false; + + /** + * Constructor. Prevent construction except by derived classes. + * + *

The expectation is that the derived class will be a non-static inner + * class in an allocator. + */ + protected AllocationReservation() { + } + + /** + * Add to the current reservation. + * + *

Adding may fail if the allocator is not allowed to consume any more space. + * + * @param nBytes the number of bytes to add + * @return true if the addition is possible, false otherwise + * @throws IllegalStateException if called after buffer() is used to allocate the reservation + */ + public boolean add(final int nBytes) { + Preconditions.checkArgument(nBytes >= 0, "nBytes(%d) < 0", nBytes); + Preconditions.checkState(!closed, "Attempt to increase reservation after reservation has been closed"); + Preconditions.checkState(!used, "Attempt to increase reservation after reservation has been used"); + + if (!reserve(nBytes)) { + return false; + } + + this.nBytes += nBytes; + return true; + } + + /** + * Requests a reservation of additional space. + * + *

The implementation of the allocator's inner class provides this. + * + * @param nBytes the amount to reserve + * @return true if the reservation can be satisfied, false otherwise + */ + protected abstract boolean reserve(int nBytes); + + /** + * Allocate a buffer whose size is the total of all the add()s made. + * + *

The allocation request can still fail, even if the amount of space + * requested is available, if the allocation cannot be made contiguously. + * + * @return the buffer, or null, if the request cannot be satisfied + * @throws IllegalStateException if called called more than once + */ + public DrillBuf buffer() { + Preconditions.checkState(!closed, "Attempt to allocate after closed"); + Preconditions.checkState(!used, "Attempt to allocate more than once"); + + final DrillBuf drillBuf = allocate(nBytes); + used = true; + return drillBuf; + } + + /** + * Allocate the a buffer of the requested size. + * + *

The implementation of the allocator's inner class provides this. + * + * @param nBytes the size of the buffer requested + * @return the buffer, or null, if the request cannot be satisfied + */ + protected abstract DrillBuf allocate(int nBytes); + + @Override + public void close() { + if (closed) { + return; + } + if (!used) { + releaseReservation(nBytes); + } + + closed = true; + } + + /** + * Return the reservation back to the allocator without having used it. + * + * @param nBytes the size of the reservation + */ + protected abstract void releaseReservation(int nBytes); + + /** + * Get the current size of the reservation (the sum of all the add()s). + * + * @return size of the current reservation + */ + public int getSize() { + return nBytes; + } + + /** + * Return whether or not the reservation has been used. + * + * @return whether or not the reservation has been used + */ + public boolean isUsed() { + return used; + } + + /** + * Return whether or not the reservation has been closed. + * + * @return whether or not the reservation has been closed + */ + public boolean isClosed() { + return closed; + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java new file mode 100644 index 00000000000..8bf2a995f5e --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java @@ -0,0 +1,31 @@ +/** + * 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.memory; + +/** + * Exception thrown when a closed BufferAllocator is used. Note + * this is an unchecked exception. + * + * @param message string associated with the cause + */ +@SuppressWarnings("serial") +public class AllocatorClosedException extends RuntimeException { + public AllocatorClosedException(String message) { + super(message); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java new file mode 100644 index 00000000000..b19ece49e21 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java @@ -0,0 +1,36 @@ +/** + * 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.memory; + +import org.apache.drill.exec.testing.ExecutionControls; + +/** + * This interface provides a means for allocator owners to inject services + * required by allocators, as well as to identify themselves for debugging purposes. + * Identification is done by overriding the implementation of + * {#link {@link Object#toString()}. + */ +public interface AllocatorOwner { + /** + * Get the current ExecutionControls from the allocator's owner. + * + * @return the current execution controls; may return null if this isn't + * possible + */ + ExecutionControls getExecutionControls(); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java new file mode 100644 index 00000000000..935f2c9e17f --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java @@ -0,0 +1,34 @@ +/** + * 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.memory; + +/** + * JMX bean interface for global allocator statistics. + */ +// TODO(cwestin) use Stats infrastructure instead of JMX beans +public interface AllocatorsStatsMXBean { + /** + * Get the maximum amount of direct memory that can be used. + * + *

This is determined by what is available, or by the drillbit + * configuration, if it specifies a value.

+ * + * @return the amount of direct memory that can be used + */ + public long getMaxDirectMemory(); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java deleted file mode 100644 index 057cfa680de..00000000000 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java +++ /dev/null @@ -1,216 +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.memory; - -import java.util.concurrent.atomic.AtomicLong; - -/** - * - * - * TODO: Fix this so that preallocation can never be released back to general pool until allocator is closed. - */ -public class AtomicRemainder { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AtomicRemainder.class); - - private static final boolean DEBUG = true; - - private final AtomicRemainder parent; - private final AtomicLong availableShared; - private final AtomicLong availablePrivate; - private final long initTotal; - private final long initShared; - private final long initPrivate; - private long limit; // An Allocator can set a variable limit less than or equal to the initTotal - private boolean hasLimit; // True for Atomic Remainders associated with a Fragment. May be true for Operator Level allocators some day. - private boolean closed = false; - private final boolean errorOnLeak; - private final boolean applyFragmentLimit; - - public AtomicRemainder(boolean errorOnLeak, AtomicRemainder parent, long max, long pre, boolean applyFragLimit) { - this.errorOnLeak = errorOnLeak; - this.parent = parent; - this.availableShared = new AtomicLong(max - pre); - this.availablePrivate = new AtomicLong(pre); - this.initTotal = max; - this.initShared = max - pre; - this.initPrivate = pre; - this.limit = max; - this.hasLimit=false; - this.applyFragmentLimit=applyFragLimit; // If this is an operator that is exempt from the fragment limit, set this to false. -// logger.info("new AtomicRemainder. a.s. {} a.p. {} hashcode {}", availableShared, availablePrivate, hashCode(), new Exception()); - } - - public long getRemainder() { - return availableShared.get() + availablePrivate.get(); - } - - public long getUsed() { - return initTotal - getRemainder(); - } - - /** - * Allow an allocator to constrain the remainder to a particular limit that is lower than the initTotal. - * If limit is larger than initTotal, then the function will do nothing and the hasLimit flag will not be set. - * @param limit - */ - public void setLimit(long limit) { - if(limit this.limit)){ - if (parent != null) { - parent.returnAllocation(size); - } - StackTraceElement[] ste = (new Throwable()).getStackTrace(); - StringBuffer sb = new StringBuffer(); - for (StackTraceElement s : ste) { - sb.append(s.toString()); - sb.append("\n"); - } - logger.warn("No more memory. Fragment limit ("+this.limit + - " bytes) reached. Trying to allocate "+size+ " bytes. "+getUsed()+" bytes already allocated.\n"+sb.toString()); - return false; - } - - // attempt to get shared memory, if fails, return false. - long outcome = availableShared.addAndGet(-size); -// assert outcome <= initShared; - if (outcome < 0) { - long newAvailableShared = availableShared.addAndGet(size); - // assert newAvailableShared <= initShared; - if (parent != null) { - parent.returnAllocation(size); - } - return false; - } else { -// if (DEBUG) -// logger.info("Get {}. a.s. {} a.p. {} hashcode {}", size, availableShared, availablePrivate, hashCode(), new Exception()); - return true; - } - - } else { - // if there is preallocated memory, use that first. - long unaccount = availablePrivate.addAndGet(-size); - if (unaccount >= 0) { -// if (DEBUG) -// logger.info("Get {}. a.s. {} a.p. {} hashcode {}", size, availableShared, availablePrivate, hashCode(), new Exception()); - return true; - } else { - - long additionalSpaceNeeded = -unaccount; - // if there is a parent allocator, check it before allocating. - if (parent != null && !parent.get(additionalSpaceNeeded, this.applyFragmentLimit)) { - // parent allocation failed, return space to private pool. - availablePrivate.getAndAdd(size); - return false; - } - - // we got space from parent pool. lets make sure we have space locally available. - long account = availableShared.addAndGet(-additionalSpaceNeeded); - if (account >= 0) { - // we were succesful, move private back to zero (since we allocated using shared). - availablePrivate.addAndGet(additionalSpaceNeeded); -// if (DEBUG) -// logger.info("Get {}. a.s. {} a.p. {} hashcode {}", size, availableShared, availablePrivate, hashCode(), new Exception()); - return true; - } else { - // we failed to get space from available shared. Return allocations to initial state. - availablePrivate.addAndGet(size); - availableShared.addAndGet(additionalSpaceNeeded); - parent.returnAllocation(additionalSpaceNeeded); - return false; - } - } - - } - - } - - /** - * Return the memory accounting to the allocation pool. Make sure to first maintain hold of the preallocated memory. - * - * @param size - */ - public void returnAllocation(long size) { - long privateSize = availablePrivate.get(); - long privateChange = Math.min(size, initPrivate - privateSize); - long sharedChange = size - privateChange; - availablePrivate.addAndGet(privateChange); - availableShared.addAndGet(sharedChange); -// if (DEBUG) -// logger.info("Return allocation {}, a.s. {} a.p. {} hashcode {}", size, availableShared, availablePrivate, hashCode(), new Exception()); - if (parent != null) { - parent.returnAllocation(sharedChange); - } - assert getUsed() <= initTotal; - } - - public void close() { - if (closed) { - logger.warn("Tried to close remainder, but it has already been closed", new Exception()); - return; - } - if (availablePrivate.get() != initPrivate || availableShared.get() != initShared) { - IllegalStateException e = new IllegalStateException( - String - .format(ERROR, initPrivate, availablePrivate.get(), initPrivate - availablePrivate.get(), initShared, availableShared.get(), initShared - availableShared.get())); - if (errorOnLeak) { - throw e; - } else { - logger.warn("Memory leaked during query.", e); - } - } - if (parent != null) { - parent.returnAllocation(initPrivate); - } - closed = true; - } - - static final String ERROR = "Failure while closing accountor. Expected private and shared pools to be set to initial values. However, one or more were not. Stats are\n\tzone\tinit\tallocated\tdelta \n\tprivate\t%d\t%d\t%d \n\tshared\t%d\t%d\t%d."; -} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java new file mode 100644 index 00000000000..e1d66e8be3e --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java @@ -0,0 +1,1673 @@ +/** + * 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.memory; + +import java.lang.management.ManagementFactory; +import java.util.Collection; +import java.util.IdentityHashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; + +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.DrillBuf; +import io.netty.buffer.PooledByteBufAllocatorL; +import io.netty.buffer.UnsafeDirectLittleEndian; + +import org.apache.drill.common.DrillAutoCloseables; +import org.apache.drill.common.HistoricalLog; +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.ops.FragmentContext; +import org.apache.drill.exec.testing.ControlsInjector; +import org.apache.drill.exec.testing.ControlsInjectorFactory; +import org.apache.drill.exec.util.AssertionUtil; +import org.apache.drill.exec.util.Pointer; +import org.slf4j.Logger; + +import com.google.common.base.Preconditions; + +public abstract class BaseAllocator implements BufferAllocator { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseAllocator.class); + private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(BaseAllocator.class); + + private static final AtomicInteger idGenerator = new AtomicInteger(0); + private static final Object ALLOCATOR_LOCK = new Object(); + + public static final String CHILD_BUFFER_INJECTION_SITE = "child.buffer"; + + private static final boolean DEBUG = AssertionUtil.isAssertionsEnabled() + || Boolean.getBoolean(ExecConstants.DEBUG_ALLOCATOR); + private static final PooledByteBufAllocatorL INNER_ALLOCATOR = PooledByteBufAllocatorL.DEFAULT; + + private long allocated; // the amount of memory this allocator has given out to its clients (including children) + private long owned; // the amount of memory this allocator has obtained from its parent + private long peakAllocated; // the most memory this allocator has given out during its lifetime + private long bufferAllocation; // the amount of memory used just for directly allocated buffers, not children + + private boolean isClosed = false; // the allocator has been closed + + private final long maxAllocation; // the maximum amount of memory this allocator will give out + private final AllocationPolicyAgent policyAgent; + private final BaseAllocator parentAllocator; + private final AllocatorOwner allocatorOwner; + protected final int id = idGenerator.incrementAndGet(); // unique ID assigned to each allocator + private final DrillBuf empty; + private final AllocationPolicy allocationPolicy; + private final InnerBufferLedger bufferLedger = new InnerBufferLedger(); + + // members used purely for debugging + private final IdentityHashMap allocatedBuffers; + private final IdentityHashMap childAllocators; + private final IdentityHashMap reservations; + private long preallocSpace; + + private final HistoricalLog historicalLog; + + /** + * Provide statistics via JMX for limiting root allocators. + */ + private class RootAllocatorStats implements RootAllocatorStatsMXBean { + @Override + public long getOwnedMemory() { + return owned; + } + + @Override + public long getAllocatedMemory() { + return allocated; + } + + @Override + public long getChildCount() { + if (childAllocators != null) { + return childAllocators.size(); + } + + return -1; // unknown + } + } + + private static BaseAllocator getBaseAllocator(final BufferAllocator bufferAllocator) { + if (!(bufferAllocator instanceof BaseAllocator)) { + throw new IllegalArgumentException("expected a BaseAllocator instance, but got a " + + bufferAllocator.getClass().getName()); + } + return (BaseAllocator) bufferAllocator; + } + + // TODO(cwestin) move allocation policy and agent to outside of allocator + private static class PerFragmentAllocationPolicy implements AllocationPolicy { + static class Globals { + private long maxBufferAllocation = 0; + private final AtomicInteger limitingRoots = new AtomicInteger(0); + } + + private final Globals globals = new Globals(); + + @Override + public AllocationPolicyAgent newAgent() { + return new PerFragmentAllocationPolicyAgent(globals); + } + } + + /** + * AllocationPolicy that allows each fragment running on a drillbit to share an + * equal amount of direct memory, regardless of whether or not those fragments + * belong to the same query. + */ + public static final AllocationPolicy POLICY_PER_FRAGMENT = new PerFragmentAllocationPolicy(); + + /** + * String name of {@link #POLICY_PER_FRAGMENT} policy. + */ + public static final String POLICY_PER_FRAGMENT_NAME = "per-fragment"; + + private static class PerFragmentAllocationPolicyAgent implements AllocationPolicyAgent { + private final PerFragmentAllocationPolicy.Globals globals; + private boolean limitingRoot; // this is a limiting root; see F_LIMITING_ROOT + + // registered MBean object name (for JMX statistics reporting) + private ObjectName objectName = null; + + PerFragmentAllocationPolicyAgent(PerFragmentAllocationPolicy.Globals globals) { + this.globals = globals; + } + + @Override + public void close() throws Exception { + if (limitingRoot) { + // now there's one fewer active root + final int rootCount = globals.limitingRoots.decrementAndGet(); + + synchronized(globals) { + /* + * If the rootCount went to zero, we don't need to worry about setting the + * maxBufferAllocation, because there aren't any allocators to reference it; + * the next allocator to get created will set it appropriately. + */ + if (rootCount != 0) { + globals.maxBufferAllocation = RootAllocator.getMaxDirect() / rootCount; + } + } + } + + if (objectName != null) { + final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); + mbs.unregisterMBean(objectName); + } + } + + @Override + public void checkNewAllocator(BufferAllocator parentAllocator, + long initReservation, long maxAllocation, int flags) { +/* + Preconditions.checkArgument(parentAllocator != null, "parent allocator can't be null"); + Preconditions.checkArgument(parentAllocator instanceof BaseAllocator, "Parent allocator must be a BaseAllocator"); +*/ + +// final BaseAllocator baseAllocator = (BaseAllocator) parentAllocator; + + // this is synchronized to protect maxBufferAllocation + synchronized(POLICY_PER_FRAGMENT) { + // initialize maxBufferAllocation the very first time we call this + if (globals.maxBufferAllocation == 0) { + globals.maxBufferAllocation = RootAllocator.getMaxDirect(); + } + + if (limitingRoot = ((flags & F_LIMITING_ROOT) != 0)) { + // figure out the new current per-allocator limit + globals.maxBufferAllocation = RootAllocator.getMaxDirect() / (globals.limitingRoots.get() + 1); + } + + if (initReservation > 0) { + if (initReservation > globals.maxBufferAllocation) { + throw new OutOfMemoryRuntimeException( + String.format("can't fulfill initReservation request at this time " + + "(initReservation = %d > maxBufferAllocation = %d)", + initReservation, globals.maxBufferAllocation)); + } + } + } + } + + @Override + public long getMemoryLimit(BufferAllocator bufferAllocator) { + synchronized(POLICY_PER_FRAGMENT) { + return globals.maxBufferAllocation; + } + } + + @Override + public void initializeAllocator(final BufferAllocator bufferAllocator) { + final BaseAllocator baseAllocator = getBaseAllocator(bufferAllocator); + + if (!limitingRoot) { + objectName = null; + } else { + globals.limitingRoots.incrementAndGet(); + + // publish management information for this allocator + try { + final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); + final RootAllocatorStatsMXBean mbean = baseAllocator.new RootAllocatorStats(); + objectName = new ObjectName("org.apache.drill.exec.memory:RootAllocator=" + baseAllocator.id); + mbs.registerMBean(mbean, objectName); + } catch(Exception e) { + logger.warn("Exception setting up RootAllocatorStatsMXBean:\n" + e); + } + } + } + + @Override + public boolean shouldReleaseToParent(final BufferAllocator bufferAllocator) { + final BaseAllocator baseAllocator = getBaseAllocator(bufferAllocator); + return (baseAllocator.bufferAllocation > globals.maxBufferAllocation); + } + } + + private static class LocalMaxAllocationPolicy implements AllocationPolicy { + // this agent is stateless, so we can always use the same one + private static final AllocationPolicyAgent AGENT = new LocalMaxAllocationPolicyAgent(); + + @Override + public AllocationPolicyAgent newAgent() { + return AGENT; + } + } + + /** + * AllocationPolicy that imposes no limits on how much direct memory fragments + * may allocate. LOCAL_MAX refers to the only limit that is enforced, which is + * the maxAllocation specified at allocators' creation. + * + *

This policy ignores the value of {@link BufferAllocator#F_LIMITING_ROOT}.

+ */ + public static final AllocationPolicy POLICY_LOCAL_MAX = new LocalMaxAllocationPolicy(); + + /** + * String name of {@link #POLICY_LOCAL_MAX} allocation policy. + */ + public static final String POLICY_LOCAL_MAX_NAME = "local-max"; + + private static class LocalMaxAllocationPolicyAgent implements AllocationPolicyAgent { + @Override + public void close() throws Exception { + } + + @Override + public void checkNewAllocator(BufferAllocator parentAllocator, + long initReservation, long maxAllocation, int flags) { + } + + @Override + public long getMemoryLimit(BufferAllocator bufferAllocator) { + final BaseAllocator baseAllocator = (BaseAllocator) bufferAllocator; + return baseAllocator.maxAllocation; + } + + @Override + public void initializeAllocator(BufferAllocator bufferAllocator) { + } + + @Override + public boolean shouldReleaseToParent(BufferAllocator bufferAllocator) { + // since there are no shared limits, release space whenever we can + return true; + } + } + + // TODO(DRILL-2698) POLICY_PER_QUERY + + protected BaseAllocator( + final BaseAllocator parentAllocator, + final AllocatorOwner allocatorOwner, + final AllocationPolicy allocationPolicy, + final long initReservation, + final long maxAllocation, + final int flags) throws OutOfMemoryRuntimeException { + Preconditions.checkArgument(allocatorOwner != null, "allocatorOwner must be non-null"); + Preconditions.checkArgument(initReservation >= 0, + "the initial reservation size must be non-negative"); + Preconditions.checkArgument(maxAllocation >= 0, + "the maximum allocation limit mjst be non-negative"); + Preconditions.checkArgument(initReservation <= maxAllocation, + "the initial reservation size must be <= the maximum allocation"); + + if (initReservation > 0) { + if (parentAllocator == null) { + throw new IllegalStateException( + "can't reserve memory without a parent allocator"); + } + } + + // check to see if we can create this new allocator (the check throws if it's not ok) + final AllocationPolicyAgent policyAgent = allocationPolicy.newAgent(); + policyAgent.checkNewAllocator(parentAllocator, initReservation, maxAllocation, flags); + + if ((initReservation > 0) && !parentAllocator.reserve(this, initReservation, 0)) { + throw new OutOfMemoryRuntimeException( + "can't fulfill initial reservation of size (unavailable from parent)" + initReservation); + } + + this.parentAllocator = parentAllocator; + this.allocatorOwner = allocatorOwner; + this.allocationPolicy = allocationPolicy; + this.policyAgent = policyAgent; + this.maxAllocation = maxAllocation; + + // the root allocator owns all of its memory; anything else just owns it's initial reservation + owned = parentAllocator == null ? maxAllocation : initReservation; + empty = DrillBuf.getEmpty(new EmptyLedger(), this); + + if (DEBUG) { + allocatedBuffers = new IdentityHashMap<>(); + childAllocators = new IdentityHashMap<>(); + reservations = new IdentityHashMap<>(); + historicalLog = new HistoricalLog(4, "allocator[%d]", id); + + historicalLog.recordEvent("created by \"%s\", owned = %d", allocatorOwner.toString(), owned); + } else { + allocatedBuffers = null; + childAllocators = null; + reservations = null; + historicalLog = null; + } + + // now that we're not in danger of throwing an exception, we can take this step + policyAgent.initializeAllocator(this); + } + + /** + * Allocators without a parent must provide an implementation of this so + * that they may reserve additional space even though they don't have a + * parent they can fall back to. + * + *

Prior to calling this, BaseAllocator has verified that this won't violate + * the maxAllocation for this allocator.

+ * + * @param nBytes the amount of space to reserve + * @param ignoreMax ignore the maximum allocation limit; + * see {@link ChildLedger#reserve(long, boolean)}. + * @return true if the request can be met, false otherwise + */ + protected boolean canIncreaseOwned(final long nBytes, final int flags) { + if (parentAllocator == null) { + return false; + } + + return parentAllocator.reserve(this, nBytes, flags); + } + + /** + * Reserve space for the child allocator from this allocator. + * + * @param childAllocator child allocator making the request, or null + * if this is not for a child + * @param nBytes how much to reserve + * @param flags one or more of RESERVE_F_* flags or'ed together + * @return true if the reservation can be satisfied, false otherwise + */ + private static final int RESERVE_F_IGNORE_MAX = 0x0001; + private boolean reserve(final BaseAllocator childAllocator, + final long nBytes, final int flags) { + Preconditions.checkArgument(nBytes >= 0, + "the number of bytes to reserve must be non-negative"); + + // we can always fulfill an empty request + if (nBytes == 0) { + return true; + } + + final boolean ignoreMax = (flags & RESERVE_F_IGNORE_MAX) != 0; + + synchronized(ALLOCATOR_LOCK) { + if (isClosed) { + throw new AllocatorClosedException(String.format("Attempt to use closed allocator[%d]", id)); + } + + final long ownAtLeast = allocated + nBytes; + // Are we allowed to hand out this much? + if (!ignoreMax && (ownAtLeast > maxAllocation)) { + return false; + } + + // do we need more from our parent first? + if (ownAtLeast > owned) { + final long needAdditional = ownAtLeast - owned; + if (!canIncreaseOwned(needAdditional, flags)) { + return false; + } + owned += needAdditional; + + if (DEBUG) { + historicalLog.recordEvent("increased owned by %d, now owned == %d", needAdditional, owned); + } + } + + if (DEBUG) { + if (owned < ownAtLeast) { + throw new IllegalStateException("don't own enough memory to satisfy request"); + } + if (allocated > owned) { + throw new IllegalStateException( + String.format("more memory allocated (%d) than owned (%d)", allocated, owned)); + } + + historicalLog.recordEvent("allocator[%d] allocated increased by nBytes == %d to %d", + id, nBytes, allocated + nBytes); + } + + allocated += nBytes; + + if (allocated > peakAllocated) { + peakAllocated = allocated; + } + + return true; + } + } + + private void releaseBytes(final long nBytes) { + Preconditions.checkArgument(nBytes >= 0, + "the number of bytes being released must be non-negative"); + + synchronized(ALLOCATOR_LOCK) { + allocated -= nBytes; + + if (DEBUG) { + historicalLog.recordEvent("allocator[%d] released nBytes == %d, allocated now %d", + id, nBytes, allocated); + } + } + } + + private void releaseBuffer(final DrillBuf drillBuf) { + Preconditions.checkArgument(drillBuf != null, + "the DrillBuf being released can't be null"); + + final ByteBuf byteBuf = drillBuf.unwrap(); + final int udleMaxCapacity = byteBuf.maxCapacity(); + + synchronized(ALLOCATOR_LOCK) { + final boolean releaseToParent = (parentAllocator != null) + && policyAgent.shouldReleaseToParent(this); + bufferAllocation -= udleMaxCapacity; + releaseBytes(udleMaxCapacity); + + /* + * Return space to our parent if our allocation is over the currently allowed amount. + */ + if (releaseToParent) { + final long canFree = owned - allocated; + parentAllocator.releaseBytes(canFree); + owned -= canFree; + + if (DEBUG) { + historicalLog.recordEvent("returned %d to parent, now owned == %d", canFree, owned); + } + } + + if (DEBUG) { + // make sure the buffer came from this allocator + final Object object = allocatedBuffers.remove(byteBuf); + if (object == null) { + historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + throw new IllegalStateException("Released buffer did not belong to this allocator"); + } + } + } + } + + private void childClosed(final BaseAllocator childAllocator) { + Preconditions.checkArgument(childAllocator != null, "child allocator can't be null"); + + if (DEBUG) { + synchronized(ALLOCATOR_LOCK) { + final Object object = childAllocators.remove(childAllocator); + if (object == null) { + childAllocator.historicalLog.logHistory(logger); + throw new IllegalStateException("Child allocator[" + childAllocator.id + + "] not found in parent allocator[" + id + "]'s childAllocators"); + } + + try { + verifyAllocator(); + } catch(Exception e) { + /* + * If there was a problem with verification, the history of the closed + * child may also be useful. + */ + logger.debug("allocator[" + id + "]: exception while closing the following child"); + childAllocator.historicalLog.logHistory(logger); + + // Continue with the verification exception throwing. + throw e; + } + } + } + } + + /** + * TODO(DRILL-2740) We use this to bypass the regular accounting for the + * empty DrillBuf, because it is treated specially at this time. Once that + * is remedied, this should be able to go away. + */ + private class EmptyLedger implements BufferLedger { + @Override + public PooledByteBufAllocatorL getUnderlyingAllocator() { + return INNER_ALLOCATOR; + } + + @Override + public void release(final DrillBuf drillBuf) { + if (DEBUG) { + if (drillBuf != empty) { + throw new IllegalStateException("The empty buffer's ledger is being used to release something else"); + } + } + } + + @Override + public BufferLedger shareWith(Pointer pDrillBuf, + BufferLedger otherLedger, BufferAllocator otherAllocator, DrillBuf drillBuf, + int index, int length, int drillBufFlags) { + throw new UnsupportedOperationException("The empty buffer can't be shared"); + } + + @Override + public boolean transferTo(BufferAllocator newAlloc, + Pointer pNewLedger, DrillBuf drillBuf) { + throw new UnsupportedOperationException("The empty buffer's ownership can't be changed"); + } + } + + private class InnerBufferLedger implements BufferLedger { + @Override + public PooledByteBufAllocatorL getUnderlyingAllocator() { + return INNER_ALLOCATOR; + } + + @Override + public void release(final DrillBuf drillBuf) { + releaseBuffer(drillBuf); + } + + @Override + public BufferLedger shareWith(final Pointer pDrillBuf, + final BufferLedger otherLedger, final BufferAllocator otherAllocator, + final DrillBuf drillBuf, final int index, final int length, final int drillBufFlags) { + final BaseAllocator baseAllocator = (BaseAllocator) otherAllocator; + synchronized(ALLOCATOR_LOCK) { + if (baseAllocator.isClosed) { + throw new AllocatorClosedException( + String.format("Attempt to use closed allocator[%d]", baseAllocator.id)); + } + + /* + * If this is called, then the buffer isn't yet shared, and should + * become so. + */ + final SharedBufferLedger sharedLedger = new SharedBufferLedger(drillBuf, BaseAllocator.this); + + // Create the new wrapping DrillBuf. + final DrillBuf newBuf = + new DrillBuf(sharedLedger, otherAllocator, drillBuf, index, length, drillBufFlags); + sharedLedger.addMapping(newBuf, baseAllocator); + + if (DEBUG) { + final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap(); + historicalLog.recordEvent("InnerBufferLedger(allocator[%d]).shareWith(..., " + + "otherAllocator[%d], drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}, ...)", + BaseAllocator.this.id, baseAllocator.id, drillBuf.getId(), + System.identityHashCode(udle)); + + final BaseAllocator drillBufAllocator = (BaseAllocator) drillBuf.getAllocator(); + if (BaseAllocator.this != drillBufAllocator) { + historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + throw new IllegalStateException(String.format( + "DrillBuf's allocator([%d]) doesn't match this(allocator[%d])", + drillBufAllocator.id, BaseAllocator.this.id)); + } + + // Replace the ledger for the existing buffer. + final BufferLedger thisLedger = allocatedBuffers.put(udle, sharedLedger); + + // If we throw any of these exceptions, we need to clean up newBuf. + if (thisLedger == null) { + newBuf.release(); + historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + throw new IllegalStateException("Buffer to be shared is unknown to the source allocator"); + } + if (thisLedger != this) { + newBuf.release(); + historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + throw new IllegalStateException("Buffer's ledger was not the one it should be"); + } + } + + pDrillBuf.value = newBuf; + return sharedLedger; + } + } + + @Override + public boolean transferTo(final BufferAllocator newAlloc, + final Pointer pNewLedger, final DrillBuf drillBuf) { + Preconditions.checkArgument(newAlloc != null, "New allocator cannot be null"); + Preconditions.checkArgument(newAlloc != BaseAllocator.this, + "New allocator is same as current"); + Preconditions.checkArgument(newAlloc instanceof BaseAllocator, + "New allocator isn't a BaseAllocator"); + Preconditions.checkArgument(pNewLedger.value != null, "Candidate new ledger can't be null"); + Preconditions.checkArgument(drillBuf != null, "DrillBuf can't be null"); + + final BaseAllocator newAllocator = (BaseAllocator) newAlloc; + synchronized(ALLOCATOR_LOCK) { + if (newAllocator.isClosed) { + throw new AllocatorClosedException( + String.format("Attempt to use closed allocator[%d]", newAllocator.id)); + } + + return BaseAllocator.transferTo(newAllocator, pNewLedger.value, drillBuf); + } + } + } + + /** + * Transfer ownership of a buffer from one allocator to another. + * + *

Assumes the allocatorLock is held.

+ * + * @param newAllocator the new allocator + * @param newLedger the new ledger to use (which could be shared) + * @param drillBuf the buffer + * @return true if the buffer's transfer didn't exceed the new owner's maximum + * allocation limit + */ + private static boolean transferTo(final BaseAllocator newAllocator, + final BufferLedger newLedger, final DrillBuf drillBuf) { + final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap(); + final int udleMaxCapacity = udle.maxCapacity(); + + synchronized(ALLOCATOR_LOCK) { + // Account for the space and track the buffer. + newAllocator.reserveForBuf(udleMaxCapacity); + + if (DEBUG) { + final Object object = newAllocator.allocatedBuffers.put(udle, newLedger); + if (object != null) { + newAllocator.historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + throw new IllegalStateException("Buffer unexpectedly found in new allocator"); + } + } + + // Remove from the old allocator. + final BaseAllocator oldAllocator = (BaseAllocator) drillBuf.getAllocator(); + oldAllocator.releaseBuffer(drillBuf); + + if (DEBUG) { + final Object object = oldAllocator.allocatedBuffers.get(udle); + if (object != null) { + oldAllocator.historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + throw new IllegalStateException("Buffer was not removed from old allocator"); + } + + oldAllocator.historicalLog.recordEvent("BaseAllocator.transferTo(otherAllocator[%d], ..., " + + "drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}) oldAllocator[%d]", + newAllocator.id, drillBuf.getId(), System.identityHashCode(drillBuf.unwrap()), + oldAllocator.id); + newAllocator.historicalLog.recordEvent("BaseAllocator.transferTo(otherAllocator[%d], ..., " + + "drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}) oldAllocator[%d]", + newAllocator.id, drillBuf.getId(), System.identityHashCode(drillBuf.unwrap()), + oldAllocator.id); + } + + return newAllocator.allocated < newAllocator.maxAllocation; + } + } + + private static class SharedBufferLedger implements BufferLedger { + private volatile BaseAllocator owningAllocator; + private final IdentityHashMap bufferMap = new IdentityHashMap<>(); + + private final HistoricalLog historicalLog; + + public SharedBufferLedger(final DrillBuf drillBuf, final BaseAllocator baseAllocator) { + if (DEBUG) { + historicalLog = new HistoricalLog(4, + "SharedBufferLedger for DrillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}", + drillBuf.getId(), System.identityHashCode(drillBuf.unwrap())); + } else { + historicalLog = null; + } + addMapping(drillBuf, baseAllocator); + owningAllocator = baseAllocator; + + if (DEBUG) { + checkBufferMap(); + } + } + + private synchronized void addMapping(final DrillBuf drillBuf, final BaseAllocator baseAllocator) { + bufferMap.put(drillBuf, baseAllocator); + + if (DEBUG) { + historicalLog.recordEvent("addMapping(DrillBuf[%d], allocator[%d])", drillBuf.getId(), baseAllocator.id); + } + } + + private synchronized void logBufferHistories(final Logger logger) { + final Set> bufsToCheck = bufferMap.entrySet(); + for(final Map.Entry mapEntry : bufsToCheck) { + final DrillBuf drillBuf = mapEntry.getKey(); + drillBuf.logHistory(logger); + } + } + + private synchronized void checkBufferMap() { + boolean foundOwner = false; + final Set> bufsToCheck = bufferMap.entrySet(); + for(final Map.Entry mapEntry : bufsToCheck) { + final DrillBuf drillBuf = mapEntry.getKey(); + final BaseAllocator bufferAllocator = mapEntry.getValue(); + + final Object object = bufferAllocator.allocatedBuffers.get(drillBuf.unwrap()); + if (bufferAllocator == owningAllocator) { + foundOwner = true; + if (object == null) { + historicalLog.logHistory(logger); + logBufferHistories(logger); + throw new IllegalStateException( + String.format("Shared buffer DrillBuf[%d] not found in owning allocator[%d]", + drillBuf.getId(), bufferAllocator.id)); + } + } else { + if (object != null) { + historicalLog.logHistory(logger); + logBufferHistories(logger); + throw new IllegalStateException( + String.format("Shared buffer DrillBuf[%d] not found in non-owning allocator[%d]", + drillBuf.getId(), bufferAllocator.id)); + + } + } + } + + if (!foundOwner && !bufferMap.isEmpty()) { + historicalLog.logHistory(logger); + logBufferHistories(logger); + owningAllocator.historicalLog.logHistory(logger); + throw new IllegalStateException( + String.format("Did not find owning allocator[%d] in bufferMap", owningAllocator.id)); + } + } + + @Override + public PooledByteBufAllocatorL getUnderlyingAllocator() { + return INNER_ALLOCATOR; + } + + @Override + public void release(final DrillBuf drillBuf) { + Preconditions.checkArgument(drillBuf != null, "drillBuf can't be null"); + + /* + * This is the only method on the shared ledger that can be entered without + * having first come through an outside method on BaseAllocator (such + * as takeOwnership() or shareOwnership()), all of which get the allocatorLock. + * Operations in the below require the allocatorLock. We also need to synchronize + * on this object to protect the bufferMap. In order to avoid a deadlock with other + * methods, we have to get the allocatorLock first, as will be done in all the + * other cases. + */ + synchronized(ALLOCATOR_LOCK) { + synchronized(this) { + final Object bufferObject = bufferMap.remove(drillBuf); + if (DEBUG) { + if (bufferObject == null) { + historicalLog.logHistory(logger, String.format("release(DrillBuf[%d])", drillBuf.getId())); + drillBuf.logHistory(logger); + throw new IllegalStateException("Buffer not found in SharedBufferLedger's buffer map"); + } + } + + /* + * If there are other buffers in the bufferMap that share this buffer's fate, + * remove them, since they are also now invalid. As we examine buffers, take note + * of any others that don't share this one's fate, but which belong to the same + * allocator; if we find any such, then we can avoid transferring ownership at this + * time. + */ + final BaseAllocator bufferAllocator = (BaseAllocator) drillBuf.getAllocator(); + final List sameAllocatorSurvivors = new LinkedList<>(); + if (!bufferMap.isEmpty()) { + /* + * We're going to be modifying bufferMap (if we find any other related buffers); + * in order to avoid getting a ConcurrentModificationException, we can't do it + * on the same iteration we use to examine the buffers, so we use an intermediate + * list to figure out which ones we have to remove. + */ + final Set> bufsToCheck = bufferMap.entrySet(); + final List sharedFateBuffers = new LinkedList<>(); + for(final Map.Entry mapEntry : bufsToCheck) { + final DrillBuf otherBuf = mapEntry.getKey(); + if (otherBuf.hasSharedFate(drillBuf)) { + sharedFateBuffers.add(otherBuf); + } else { + final BaseAllocator otherAllocator = mapEntry.getValue(); + if (otherAllocator == bufferAllocator) { + sameAllocatorSurvivors.add(otherBuf); + } + } + } + + final int nSharedFate = sharedFateBuffers.size(); + if (nSharedFate > 0) { + final int[] sharedIds = new int[nSharedFate]; + int index = 0; + for(final DrillBuf bufToRemove : sharedFateBuffers) { + sharedIds[index++] = bufToRemove.getId(); + bufferMap.remove(bufToRemove); + } + + if (DEBUG) { + final StringBuilder sb = new StringBuilder(); + for(final DrillBuf bufToRemove : sharedFateBuffers) { + sb.append(String.format("DrillBuf[%d], ", bufToRemove.getId())); + } + sb.setLength(sb.length() - 2); // Chop off the trailing comma and space. + historicalLog.recordEvent("removed shared fate buffers " + sb.toString()); + } + } + } + + if (sameAllocatorSurvivors.isEmpty()) { + /* + * If that was the owning allocator, then we need to transfer ownership to + * another allocator (any one) that is part of the sharing set. + * + * When we release the buffer back to the allocator, release the root buffer, + */ + if (bufferAllocator == owningAllocator) { + if (bufferMap.isEmpty()) { + /* + * There are no other allocators available to transfer to, so + * release the space to the owner. + */ + bufferAllocator.releaseBuffer(drillBuf); + } else { + // Pick another allocator, and transfer ownership to that. + final Collection allocators = bufferMap.values(); + final Iterator allocatorIter = allocators.iterator(); + if (!allocatorIter.hasNext()) { + historicalLog.logHistory(logger); + throw new IllegalStateException("Shared ledger buffer map is non-empty, but not iterable"); + } + final BaseAllocator nextAllocator = allocatorIter.next(); + BaseAllocator.transferTo(nextAllocator, this, drillBuf); + owningAllocator = nextAllocator; + + if (DEBUG) { + if (owningAllocator == bufferAllocator) { + historicalLog.logHistory(logger); + owningAllocator.historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + throw new IllegalStateException("Shared buffer release transfer to same owner"); + } + + final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap(); + final Object oldObject = bufferAllocator.allocatedBuffers.get(udle); + if (oldObject != null) { + historicalLog.logHistory(logger); + bufferAllocator.historicalLog.logHistory(logger); + owningAllocator.historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + + throw new IllegalStateException("Inconsistent shared buffer release state (old owner)"); + } + + final Object newObject = owningAllocator.allocatedBuffers.get(udle); + if (newObject == null) { + historicalLog.logHistory(logger); + bufferAllocator.historicalLog.logHistory(logger); + owningAllocator.historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + + throw new IllegalStateException("Inconsistent shared buffer release state (new owner)"); + } + } + } + } + } + } + } + + if (DEBUG) { + checkBufferMap(); + } + } + + @Override + public BufferLedger shareWith(final Pointer pDrillBuf, + final BufferLedger otherLedger, final BufferAllocator otherAllocator, + final DrillBuf drillBuf, final int index, final int length, final int drillBufFlags) { + final BaseAllocator baseAllocator = (BaseAllocator) otherAllocator; + if (baseAllocator.isClosed) { + throw new AllocatorClosedException( + String.format("Attempt to use closed allocator[%d]", baseAllocator.id)); + } + + synchronized(ALLOCATOR_LOCK) { + /* + * This buffer is already shared, but we want to add more sharers. + * + * Create the new wrapper. + */ + final DrillBuf newBuf = new DrillBuf(this, otherAllocator, drillBuf, index, length, drillBufFlags); + if (DEBUG) { + final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap(); + baseAllocator.historicalLog.recordEvent("SharedBufferLedger.shareWith(..., otherAllocator[%d], " + + "drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}, ...)", + baseAllocator.id, drillBuf.getId(), System.identityHashCode(udle)); + + // Make sure the current ownership is still correct. + final Object object = owningAllocator.allocatedBuffers.get(udle); // This may not be protectable w/o ALLOCATOR_LOCK. + if (object == null) { + newBuf.release(); + historicalLog.logHistory(logger); + owningAllocator.historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + throw new IllegalStateException("Buffer not found in owning allocator"); + } + } + + addMapping(newBuf, baseAllocator); + pDrillBuf.value = newBuf; + + if (DEBUG) { + checkBufferMap(); + } + + return this; + } + } + + @Override + public boolean transferTo(final BufferAllocator newAlloc, + final Pointer pNewLedger, final DrillBuf drillBuf) { + Preconditions.checkArgument(newAlloc != null, "New allocator cannot be null"); + Preconditions.checkArgument(newAlloc instanceof BaseAllocator, + "New allocator isn't a BaseAllocator"); + Preconditions.checkArgument(pNewLedger.value != null, "Candidate new ledger can't be null"); + Preconditions.checkArgument(drillBuf != null, "DrillBuf can't be null"); + + final BaseAllocator newAllocator = (BaseAllocator) newAlloc; + if (newAllocator.isClosed) { + throw new AllocatorClosedException(String.format( + "Attempt to use closed allocator[%d]", newAllocator.id)); + } + + // This doesn't need the ALLOCATOR_LOCK, because it will already be held. + synchronized(this) { + try { + // Modify the buffer mapping to reflect the virtual transfer. + final BaseAllocator oldAllocator = bufferMap.put(drillBuf, newAllocator); + if (oldAllocator == null) { + final BaseAllocator bufAllocator = (BaseAllocator) drillBuf.getAllocator(); + historicalLog.logHistory(logger); + bufAllocator.historicalLog.logHistory(logger); + drillBuf.logHistory(logger); + throw new IllegalStateException("No previous entry in SharedBufferLedger for drillBuf"); + } + + // Whatever happens, this is the new ledger. + pNewLedger.value = this; + + /* + * If the oldAllocator was the owner, then transfer ownership to the new allocator. + */ + if (oldAllocator == owningAllocator) { + owningAllocator = newAllocator; + return BaseAllocator.transferTo(newAllocator, this, drillBuf); + } + + // Even though we didn't do a real transfer, tell if it would have fit the limit. + final int udleMaxCapacity = drillBuf.unwrap().maxCapacity(); + return newAllocator.allocated + udleMaxCapacity < newAllocator.maxAllocation; + } finally { + if (DEBUG) { + checkBufferMap(); + } + } + } + } + } + + @Override + public DrillBuf buffer(int size) { + return buffer(size, size); + } + + private static String createErrorMsg(final BufferAllocator allocator, final int size) { + return String.format("Unable to allocate buffer of size %d due to memory limit. Current allocation: %d", + size, allocator.getAllocatedMemory()); + } + + @Override + public DrillBuf buffer(final int minSize, final int maxSize) { + Preconditions.checkArgument(minSize >= 0, + "the minimimum requested size must be non-negative"); + Preconditions.checkArgument(maxSize >= 0, + "the maximum requested size must be non-negative"); + Preconditions.checkArgument(minSize <= maxSize, + "the minimum requested size must be <= the maximum requested size"); + + if (DEBUG) { + injector.injectUnchecked(allocatorOwner.getExecutionControls(), CHILD_BUFFER_INJECTION_SITE); + } + + // we can always return an empty buffer + if (minSize == 0) { + return getEmpty(); + } + + synchronized(ALLOCATOR_LOCK) { + // Don't allow the allocation if it will take us over the limit. + final long allocatedWas = allocated; + if (!reserve(null, maxSize, 0)) { + throw new OutOfMemoryRuntimeException(createErrorMsg(this, minSize)); + } + + final long reserved = allocated - allocatedWas; + assert reserved == maxSize; + + final UnsafeDirectLittleEndian buffer = INNER_ALLOCATOR.directBuffer(minSize, maxSize); + final int actualSize = buffer.maxCapacity(); + if (actualSize > maxSize) { + final int extraSize = actualSize - maxSize; + reserve(null, extraSize, RESERVE_F_IGNORE_MAX); + } + + final DrillBuf wrapped = new DrillBuf(bufferLedger, this, buffer); + buffer.release(); // Should have been retained by the DrillBuf constructor. + assert buffer.refCnt() == 1 : "buffer was not retained by DrillBuf"; + assert allocated <= owned : "allocated more memory than owned"; + + bufferAllocation += maxSize; + if (allocated > peakAllocated) { + peakAllocated = allocated; + } + + if (allocatedBuffers != null) { + allocatedBuffers.put(buffer, bufferLedger); + } + + return wrapped; + } + } + + @Override + public ByteBufAllocator getUnderlyingAllocator() { + return INNER_ALLOCATOR; + } + + @Override + public BufferAllocator newChildAllocator(final AllocatorOwner allocatorOwner, + final long initReservation, final long maxAllocation, final int flags) { + synchronized(ALLOCATOR_LOCK) { + final BaseAllocator childAllocator = + new ChildAllocator(this, allocatorOwner, allocationPolicy, + initReservation, maxAllocation, flags); + + if (DEBUG) { + childAllocators.put(childAllocator, childAllocator); + historicalLog.recordEvent("allocator[%d] created new child allocator[%d]", + id, childAllocator.id); + } + + return childAllocator; + } + } + + @Override + public BufferAllocator getChildAllocator(FragmentContext fragmentContext, + final long initialReservation, final long maximumAllocation, + final boolean applyFragmentLimit) { + +/* TODO(cwestin) + if (fragmentContext != null) { + throw new IllegalArgumentException("fragmentContext is non-null"); + } + + if (!applyFragmentLimit) { + throw new IllegalArgumentException("applyFragmentLimit is false"); + } + */ + + return newChildAllocator(allocatorOwner, initialReservation, maximumAllocation, + (applyFragmentLimit ? F_LIMITING_ROOT : 0)); + } + + /** + * Reserve space for a DrillBuf for an ownership transfer. + * + * @param drillBuf the buffer to reserve space for + */ + private void reserveForBuf(final int maxCapacity) { + final boolean reserved = reserve(null, maxCapacity, RESERVE_F_IGNORE_MAX); + if (DEBUG) { + if (!reserved) { + throw new IllegalStateException("reserveForBuf() failed"); + } + } + } + + @Override + public boolean takeOwnership(final DrillBuf drillBuf) { + // If already owned by this, there's nothing to do. + if (this == drillBuf.getAllocator()) { + return true; + } + + synchronized(ALLOCATOR_LOCK) { + return drillBuf.transferTo(this, bufferLedger); + } + } + + @Override + public boolean shareOwnership(final DrillBuf drillBuf, final Pointer bufOut) { + synchronized(ALLOCATOR_LOCK) { + bufOut.value = drillBuf.shareWith(bufferLedger, this, 0, drillBuf.capacity()); + return allocated < maxAllocation; + } + } + + /* + * It's not clear why we'd allow anyone to set their own limit, need to see why this is used; + * this also doesn't make sense when the limits are constantly shifting, nor for other + * allocation policies. + */ + @Deprecated + @Override + public void setFragmentLimit(long fragmentLimit) { + throw new UnsupportedOperationException("unimplemented:BaseAllocator.setFragmentLimit()"); + } + + /** + * Get the fragment limit. This was originally meant to be the maximum amount + * of memory the currently running fragment (which owns this allocator or + * its ancestor) may use. Note that the value may vary up and down over time + * as fragments come and go on the node. + * + *

This is deprecated because the concept is not entirely stable. This + * only makes sense for one particular memory allocation policy, which is the + * one that sets limits on what fragments on a node may use by dividing up all + * the memory evenly between all the fragments (see {@see #POLICY_PER_FRAGMENT}). + * Other allocation policies, such as the one that limits memory on a + * per-query-per-node basis, wouldn't have a value for this. But we need to have + * something until we figure out what to eplace this with because it is used by + * some operators (such as ExternalSortBatch) to determine how much memory they + * can use before they have to spill to disk.

+ * + * @return the fragment limit + */ + @Deprecated + @Override + public long getFragmentLimit() { + return policyAgent.getMemoryLimit(this); + } + + @Override + public void close() throws Exception { + /* + * Some owners may close more than once because of complex cleanup and shutdown + * procedures. + */ + if (isClosed) { + return; + } + + synchronized(ALLOCATOR_LOCK) { + if (DEBUG) { + verifyAllocator(); + + // are there outstanding child allocators? + if (!childAllocators.isEmpty()) { + for(final BaseAllocator childAllocator : childAllocators.keySet()) { + if (childAllocator.isClosed) { + logger.debug(String.format( + "Closed child allocator[%d] on parent allocator[%d]'s child list", + childAllocator.id, id)); + } + } + + historicalLog.logHistory(logger); + logChildren(); + + throw new IllegalStateException( + String.format("Allocator[%d] closed with outstanding child allocators", id)); + } + + // are there outstanding buffers? + final int allocatedCount = allocatedBuffers.size(); + if (allocatedCount > 0) { + historicalLog.logHistory(logger); + logBuffers(); + + throw new IllegalStateException( + String.format("Allocator[%d] closed with outstanding buffers allocated (%d)", + id, allocatedCount)); + } + + if (reservations.size() != 0) { + historicalLog.logHistory(logger); + logReservations(ReservationsLog.ALL); + + throw new IllegalStateException( + String.format("Allocator closed with outstanding reservations (%d)", reservations.size())); + } + + /* TODO(DRILL-2740) + // We should be the only client holding a reference to empty now. + final int emptyRefCnt = empty.refCnt(); + if (emptyRefCnt != 1) { + final String msg = "empty buffer refCnt() == " + emptyRefCnt + " (!= 1)"; + final StringWriter stringWriter = new StringWriter(); + stringWriter.write(msg); + stringWriter.write('\n'); + empty.writeState(stringWriter); + logger.debug(stringWriter.toString()); + throw new IllegalStateException(msg); + } + */ + } + + // Is there unaccounted-for outstanding allocation? + if (allocated > 0) { + if (DEBUG) { + historicalLog.logHistory(logger); + } + throw new IllegalStateException( + String.format("Unaccounted for outstanding allocation (%d)", allocated)); + } + + // Any unclaimed reservations? + if (preallocSpace > 0) { + if (DEBUG) { + historicalLog.logHistory(logger); + } + throw new IllegalStateException( + String.format("Unclaimed preallocation space (%d)", preallocSpace)); + } + + /* + * Let go of the empty buffer. If the allocator has been closed more than once, + * this may not be necessary, so check to avoid illegal states. + */ + final int emptyCount = empty.refCnt(); + if (emptyCount > 0) { + empty.release(emptyCount); + } + + DrillAutoCloseables.closeNoChecked(policyAgent); + + // Inform our parent allocator that we've closed. + if (parentAllocator != null) { + parentAllocator.releaseBytes(owned); + owned = 0; + parentAllocator.childClosed(this); + } + + if (DEBUG) { + historicalLog.recordEvent("closed"); + } + + isClosed = true; + } + } + + /** + * Log information about child allocators; only works if DEBUG + */ + private void logChildren() { + logger.debug(String.format("allocator[%d] open child allocators BEGIN", id)); + final Set allocators = childAllocators.keySet(); + for(final BaseAllocator childAllocator : allocators) { + childAllocator.historicalLog.logHistory(logger); + } + logger.debug(String.format("allocator[%d] open child allocators END", id)); + } + + private void logBuffers() { + final StringBuilder sb = new StringBuilder(); + final Set udleSet = allocatedBuffers.keySet(); + + sb.append("allocator["); + sb.append(Integer.toString(id)); + sb.append("], "); + sb.append(Integer.toString(udleSet.size())); + sb.append(" allocated buffers\n"); + + for(final UnsafeDirectLittleEndian udle : udleSet) { + sb.append(udle.toString()); + sb.append("[identityHashCode == "); + sb.append(Integer.toString(System.identityHashCode(udle))); + sb.append("]\n"); + + final Collection drillBufs = DrillBuf.unwrappedGet(udle); + for(DrillBuf drillBuf : drillBufs) { + drillBuf.logHistory(logger); + } + } + + logger.debug(sb.toString()); + } + + private enum ReservationsLog { + ALL, + UNUSED, + } + + private void logReservations(final ReservationsLog reservationsLog) { + final StringBuilder sb = new StringBuilder(); + sb.append(String.format("allocator[%d] reservations BEGIN", id)); + + final Set reservations = this.reservations.keySet(); + for(final Reservation reservation : reservations) { + if ((reservationsLog == ReservationsLog.ALL) + || ((reservationsLog == ReservationsLog.UNUSED) && (!reservation.isUsed()))) { + reservation.writeHistoryToBuilder(sb); + } + } + + sb.append(String.format("allocator[%d] reservations END", id)); + + logger.debug(sb.toString()); + } + + @Override + public long getAllocatedMemory() { + return allocated; + } + + @Override + public int getId() { + return id; + } + + @Override + public long getPeakMemoryAllocation() { + return peakAllocated; + } + + @Override + public DrillBuf getEmpty() { + empty.retain(1); + // TODO(DRILL-2740) update allocatedBuffers + return empty; + } + + private class Reservation extends AllocationReservation { + private final HistoricalLog historicalLog; + + public Reservation() { + if (DEBUG) { + historicalLog = new HistoricalLog("Reservation[allocator[%d], %d]", id, System.identityHashCode(this)); + historicalLog.recordEvent("created"); + synchronized(ALLOCATOR_LOCK) { + reservations.put(this, this); + } + } else { + historicalLog = null; + } + } + + @Override + public void close() { + if (DEBUG) { + if (!isClosed()) { + final Object object; + synchronized(ALLOCATOR_LOCK) { + object = reservations.remove(this); + } + if (object == null) { + final StringBuilder sb = new StringBuilder(); + writeHistoryToBuilder(sb); + + logger.debug(sb.toString()); + throw new IllegalStateException( + String.format("Didn't find closing reservation[%d]", System.identityHashCode(this))); + } + + historicalLog.recordEvent("closed"); + } + } + + super.close(); + } + + @Override + protected boolean reserve(int nBytes) { + final boolean reserved; + synchronized(ALLOCATOR_LOCK) { + reserved = BaseAllocator.this.reserve(null, nBytes, 0); + if (reserved) { + preallocSpace += nBytes; + } + } + + if (DEBUG) { + historicalLog.recordEvent("reserve(%d) => %s", nBytes, Boolean.toString(reserved)); + } + + return reserved; + } + + @Override + protected DrillBuf allocate(int nBytes) { + /* + * The reservation already added the requested bytes to the + * allocators owned and allocated bytes via reserve(). This + * ensures that they can't go away. But when we ask for the buffer + * here, that will add to the allocated bytes as well, so we need to + * return the same number back to avoid double-counting them. + */ + synchronized(ALLOCATOR_LOCK) { + BaseAllocator.this.allocated -= nBytes; + final DrillBuf drillBuf = BaseAllocator.this.buffer(nBytes); + preallocSpace -= nBytes; + + if (DEBUG) { + historicalLog.recordEvent("allocate() => %s", + drillBuf == null ? "null" : String.format("DrillBuf[%d]", drillBuf.getId())); + } + + return drillBuf; + } + } + + @Override + protected void releaseReservation(int nBytes) { + synchronized(ALLOCATOR_LOCK) { + releaseBytes(nBytes); + preallocSpace -= nBytes; + } + + if (DEBUG) { + historicalLog.recordEvent("releaseReservation(%d)", nBytes); + } + } + + private String getState() { + return String.format("size == %d, isUsed == %s", getSize(), Boolean.toString(isUsed())); + } + + private void writeToBuilder(final StringBuilder sb) { + sb.append(String.format("reservation[%d]: ", System.identityHashCode(this))); + sb.append(getState()); + } + + /** + * Only works for DEBUG + * + * @param sb builder to write to + */ + private void writeHistoryToBuilder(final StringBuilder sb) { + historicalLog.buildHistory(sb, getState()); + } + } + + @Override + public AllocationReservation newReservation() { + return new Reservation(); + } + + /** + * Verifies the accounting state of the allocator. Only works for DEBUG. + * + * @throws IllegalStateException when any problems are found + */ + protected void verifyAllocator() { + final IdentityHashMap buffersSeen = new IdentityHashMap<>(); + verifyAllocator(buffersSeen); + } + + /** + * Verifies the accounting state of the allocator. Only works for DEBUG. + * + *

This overload is used for recursive calls, allowing for checking that DrillBufs are unique + * across all allocators that are checked.

+ * + * @param buffersSeen a map of buffers that have already been seen when walking a tree of allocators + * @throws IllegalStateException when any problems are found + */ + protected void verifyAllocator( + final IdentityHashMap buffersSeen) { + synchronized(ALLOCATOR_LOCK) { + // verify purely local accounting + if (allocated > owned) { + historicalLog.logHistory(logger); + throw new IllegalStateException("Allocator (id = " + id + ") has allocated more than it owns"); + } + + // the empty buffer should still be empty + final long emptyCapacity = empty.maxCapacity(); + if (emptyCapacity != 0) { + throw new IllegalStateException("empty buffer maxCapacity() == " + emptyCapacity + " (!= 0)"); + } + + // The remaining tests can only be performed if we're in debug mode. + if (!DEBUG) { + return; + } + + // verify my direct descendants + final Set childSet = childAllocators.keySet(); + for(final BaseAllocator childAllocator : childSet) { + childAllocator.verifyAllocator(buffersSeen); + } + + /* + * Verify my relationships with my descendants. + * + * The sum of direct child allocators' owned memory must be <= my allocated memory; + * my allocated memory also includes DrillBuf's directly allocated by me. + */ + long childTotal = 0; + for(final BaseAllocator childAllocator : childSet) { + childTotal += childAllocator.owned; + } + if (childTotal > allocated) { + historicalLog.logHistory(logger); + logger.debug("allocator[" + id + "] child event logs BEGIN"); + for(final BaseAllocator childAllocator : childSet) { + childAllocator.historicalLog.logHistory(logger); + } + logger.debug("allocator[" + id + "] child event logs END"); + throw new IllegalStateException( + "Child allocators own more memory (" + childTotal + ") than their parent (id = " + + id + " ) has allocated (" + allocated + ')'); + } + + // Furthermore, the amount I've allocated should be that plus buffers I've allocated. + long bufferTotal = 0; + final Set udleSet = allocatedBuffers.keySet(); + for(final UnsafeDirectLittleEndian udle : udleSet) { + /* + * Even when shared, DrillBufs are rewrapped, so we should never see the same + * instance twice. + */ + final BaseAllocator otherOwner = buffersSeen.get(udle); + if (otherOwner != null) { + throw new IllegalStateException("This allocator's drillBuf already owned by another allocator"); + } + buffersSeen.put(udle, this); + + bufferTotal += udle.maxCapacity(); + } + + // Preallocated space has to be accounted for + final Set reservationSet = reservations.keySet(); + long reservedTotal = 0; + for(final Reservation reservation : reservationSet) { + if (!reservation.isUsed()) { + reservedTotal += reservation.getSize(); + } + } + if (reservedTotal != preallocSpace) { + logReservations(ReservationsLog.UNUSED); + + throw new IllegalStateException( + String.format("This allocator's reservedTotal(%d) doesn't match preallocSpace (%d)", + reservedTotal, preallocSpace)); + } + + if (bufferTotal + reservedTotal + childTotal != allocated) { + final StringBuilder sb = new StringBuilder(); + sb.append("allocator["); + sb.append(Integer.toString(id)); + sb.append("]\nallocated: "); + sb.append(Long.toString(allocated)); + sb.append(" allocated - (bufferTotal + reservedTotal + childTotal): "); + sb.append(Long.toString(allocated - (bufferTotal + reservedTotal + childTotal))); + sb.append('\n'); + + if (bufferTotal != 0) { + sb.append("buffer total: "); + sb.append(Long.toString(bufferTotal)); + sb.append('\n'); + dumpBuffers(sb, udleSet); + } + + if (childTotal != 0) { + sb.append("child total: "); + sb.append(Long.toString(childTotal)); + sb.append('\n'); + + for(final BaseAllocator childAllocator : childSet) { + sb.append("child allocator["); + sb.append(Integer.toString(childAllocator.id)); + sb.append("] owned "); + sb.append(Long.toString(childAllocator.owned)); + sb.append('\n'); + } + } + + if (reservedTotal != 0) { + sb.append(String.format("reserved total : ", reservedTotal)); + for(final Reservation reservation : reservationSet) { + reservation.writeToBuilder(sb); + sb.append('\n'); + } + } + + logger.debug(sb.toString()); + throw new IllegalStateException(String.format( + "allocator[%d]: buffer space (%d) + prealloc space (%d) + child space (%d) != allocated (%d)", + id, bufferTotal, reservedTotal, childTotal, allocated)); + } + } + } + + private void dumpBuffers(final StringBuilder sb, final Set udleSet) { + for(final UnsafeDirectLittleEndian udle : udleSet) { + sb.append("UnsafeDirectLittleEndian[dentityHashCode == "); + sb.append(Integer.toString(System.identityHashCode(udle))); + sb.append("] size "); + sb.append(Integer.toString(udle.maxCapacity())); + sb.append('\n'); + } + } + + public static boolean isDebug() { + return DEBUG; + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java index b01534e9d11..2f24807bce8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java @@ -19,118 +19,169 @@ import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.DrillBuf; -import io.netty.buffer.UnsafeDirectLittleEndian; - -import java.io.Closeable; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.util.Pointer; +// TODO(cwestin) remove deprecated items /** - * Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods. Also allows inser + * Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods. */ -public interface BufferAllocator extends Closeable { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BufferAllocator.class); - +public interface BufferAllocator extends AutoCloseable { /** * Allocate a new or reused buffer of the provided size. Note that the buffer may technically be larger than the - * requested size for rounding purposes. However, the buffers capacity will be set to the configured size. + * requested size for rounding purposes. However, the buffer's capacity will be set to the configured size. * - * @param size - * The size in bytes. - * @return A new ByteBuf. + * @param size The size in bytes. + * @return a new DrillBuf, or null if the request can't be satisfied * @throws OutOfMemoryRuntimeException if buffer cannot be allocated */ - public abstract DrillBuf buffer(int size); + public DrillBuf buffer(int size); /** * Allocate a new or reused buffer within provided range. Note that the buffer may technically be larger than the - * requested size for rounding purposes. However, the buffers capacity will be set to the configured size. + * requested size for rounding purposes. However, the buffer's capacity will be set to the configured size. * * @param minSize The minimum size in bytes. * @param maxSize The maximum size in bytes. - * @return A new ByteBuf. + * @return a new DrillBuf, or null if the request can't be satisfied * @throws OutOfMemoryRuntimeException if buffer cannot be allocated */ - public abstract DrillBuf buffer(int minSize, int maxSize); + public DrillBuf buffer(int minSize, int maxSize); - public abstract ByteBufAllocator getUnderlyingAllocator(); + /** + * Returns the allocator this allocator falls back to when it needs more memory. + * + * @return the underlying allocator used by this allocator + */ + public ByteBufAllocator getUnderlyingAllocator(); /** * Create a child allocator nested below this one. * - * @param context - owning fragment for this allocator + * @param context - the owner or this allocator * @param initialReservation - specified in bytes * @param maximumReservation - specified in bytes * @param applyFragmentLimit - flag to conditionally enable fragment memory limits * @return - a new buffer allocator owned by the parent it was spawned from - * @throws OutOfMemoryException - when off-heap memory has been exhausted */ - public abstract BufferAllocator getChildAllocator(FragmentContext context, long initialReservation, - long maximumReservation, boolean applyFragmentLimit) throws OutOfMemoryException; + @Deprecated + public BufferAllocator getChildAllocator(FragmentContext context, long initialReservation, + long maximumReservation, boolean applyFragmentLimit); /** - * Take over ownership of fragment accounting. Always takes over ownership. - * @param buf - * @return false if over allocation. + * Flag: this allocator is a limiting sub-tree root, meaning that the maxAllocation for + * it applies to all its descendant child allocators. In low memory situations, the limits + * for sub-tree roots may be adjusted down so that they evenly share the total amount of + * direct memory across all the sub-tree roots. */ - public boolean takeOwnership(DrillBuf buf) ; + public final static int F_LIMITING_ROOT = 0x0001; /** - * Take over ownership of fragment accounting. Always takes over ownership. - * @param buf - * @return false if over allocation. + * Create a new child allocator. + * + * @param allocatorOwner the allocator owner + * @param initReservation the initial space reservation (obtained from this allocator) + * @param maxAllocation maximum amount of space the new allocator can allocate + * @param flags one or more of BufferAllocator.F_* flags + * @return the new allocator, or null if it can't be created */ - public boolean takeOwnership(DrillBuf buf, Pointer bufOut); - - public PreAllocator getNewPreAllocator(); - - //public void addFragmentContext(FragmentContext c); + public BufferAllocator newChildAllocator(AllocatorOwner allocatorOwner, + long initReservation, long maxAllocation, int flags); /** - * For Top Level Allocators. Reset the fragment limits for all allocators + * Take over ownership of the given buffer, adjusting accounting accordingly. + * This allocator always takes over ownership. + * + * @param buf the buffer to take over + * @return false if over allocation */ - public void resetFragmentLimits(); + public boolean takeOwnership(DrillBuf buf); /** - * For Child allocators to set the Fragment limit for the corresponding fragment allocator. - * @param l the new fragment limit + * Share ownership of a buffer between allocators. + * + * @param buf the buffer + * @param bufOut a new DrillBuf owned by this allocator, but sharing the same underlying buffer + * @return false if over allocation. */ - public void setFragmentLimit(long l); - - public long getFragmentLimit(); - + public boolean shareOwnership(DrillBuf buf, Pointer bufOut); /** * Not thread safe. * * WARNING: unclaimed pre-allocations leak memory. If you call preAllocate(), you must * make sure to ultimately try to get the buffer and release it. + * + * For Child allocators to set their Fragment limits. + * + * @param fragmentLimit the new fragment limit */ - public interface PreAllocator { - public boolean preAllocate(int bytes); - - public DrillBuf getAllocation(); - } + @Deprecated // happens automatically, and via allocation policies + public void setFragmentLimit(long fragmentLimit); /** - * @param bytes - * @return + * Returns the current fragment limit. + * + * @return the current fragment limit + */ + /* + * TODO should be replaced with something more general because of + * the availability of multiple allocation policies + * + * TODO We should also have a getRemainingMemory() so operators + * can query how much more is left to allocate. That could be + * tricky. */ + @Deprecated + public long getFragmentLimit(); /** + * Return a unique Id for an allocator. Id's may be recycled after + * a long period of time. + * + *

Primary use for this is for debugging output.

* + * @return the allocator's id */ + public int getId(); /** * Close and release all buffers generated from this buffer pool. + * + *

When assertions are on, complains if there are any outstanding buffers; to avoid + * that, release all buffers before the allocator is closed. */ @Override - public abstract void close(); + public void close() throws Exception; - public abstract long getAllocatedMemory(); + /** + * Returns the amount of memory currently allocated from this allocator. + * + * @return the amount of memory currently allocated + */ + public long getAllocatedMemory(); - public abstract long getPeakMemoryAllocation(); + /** + * Returns the peak amount of memory allocated from this allocator. + * + * @return the peak amount of memory allocated + */ + public long getPeakMemoryAllocation(); + /** + * Returns an empty DrillBuf. + * + * @return an empty DrillBuf + */ public DrillBuf getEmpty(); + + /** + * Create an allocation reservation. A reservation is a way of building up + * a request for a buffer whose size is not known in advance. See + * {@see AllocationReservation}. + * + * @return the newly created reservation + */ + public AllocationReservation newReservation(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferLedger.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferLedger.java new file mode 100644 index 00000000000..d6499407743 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferLedger.java @@ -0,0 +1,94 @@ +/** + * 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.memory; + +import org.apache.drill.exec.util.Pointer; + +import io.netty.buffer.DrillBuf; +import io.netty.buffer.PooledByteBufAllocatorL; + +/** + * BufferLedger is an interface meant to facility the private + * exchange of information between a DrillBuf and its owning + * allocator. To that end, a number of DrillBuf constructors + * and methods take a BufferLedger as an argument, yet there + * are no public implementations of BufferLedger; they all + * come from inner classes implemented by allocators, ensuring + * that allocators can give DrillBufs the access they need when + * they are created or asked to perform complex operations such + * as ownership sharing or transfers. + */ +public interface BufferLedger { + /** + * Get the underlying pooled allocator used by this ledger's + * allocator. + * + *

This is usually used to create the shared singleton + * empty buffer. Don't use it to create random buffers, because + * they won't be tracked, and we won't be able to find leaks.

+ * + * @return the underlying pooled allocator + */ + public PooledByteBufAllocatorL getUnderlyingAllocator(); + + /** + * Return a buffer's memory to the allocator. + * + * @param drillBuf the DrillBuf that was freed + */ + public void release(DrillBuf drillBuf); + + /** + * Share ownership of a buffer with another allocator. As far as reader + * and writer index positions go, this acts like a new slice that is owned + * by the target allocator, but which has it's own lifetime (i.e., it doesn't + * share the fate of the original buffer, unlike real slices). + * + * @param pDrillBuf returns the new DrillBuf that is shared + * @param otherLedger the ledger the new DrillBuf should use + * @param otherAllocator the new allocator-owner + * @param drillBuf the original DrillBuf to be shared + * @param index the starting index to be shared (as for slicing) + * @param length the length to be shared (as for slicing) + * @param drillBufFlags private flags passed through from the allocator + * (this call originates with a call to BufferAllocator.shareOwnership()). + * @return the ledger the calling DrillBuf must use from this point forward; + * this may not match it's original ledger, as allocators provide multiple + * implementations of ledgers to cope with sharing and slicing + */ + public BufferLedger shareWith(Pointer pDrillBuf, + BufferLedger otherLedger, BufferAllocator otherAllocator, + DrillBuf drillBuf, int index, int length, int drillBufFlags); + + /** + * Transfer the ownership of a buffer to another allocator. This doesn't change + * any of the buffer's reader or writer positions or size, just which allocator + * owns it. The reference count stays the same. + * + * @param newAlloc the new allocator (the one to transfer to) + * @param pNewLedger a Pointer<> initialized with a candidate ledger; this + * may be used, or it may not, depending on the sharing state of the buffer. + * The caller is required to use whatever ledger is in pNewLedger on return + * @param drillBuf the buffer to transfer + * @return true if the transfer kept the target allocator within its maximum + * allocation limit; false if the allocator now owns more memory than its + * creation-time maximum + */ + public boolean transferTo(final BufferAllocator newAlloc, + final Pointer pNewLedger, final DrillBuf drillBuf); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java new file mode 100644 index 00000000000..339f75497c4 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java @@ -0,0 +1,54 @@ +/** + * 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.memory; + +import org.apache.drill.exec.testing.ExecutionControls; + +/** + * An {@link AllocatorOwner} implementation that supports linking ownership to + * a parent object. This can be convenient for identification purposes, when the + * parent of the allocator's owner is a better handle for debugging. + * + *

The implementation of {@link #getExecutionControls()} returns the childOwner's + * response to getExecutionControls().

+ */ +public class ChainedAllocatorOwner implements AllocatorOwner { + private final AllocatorOwner childOwner; + private final AllocatorOwner parentOwner; + + /** + * Constructor. + * + * @param childOwner the owner of the allocator + * @param parentOwner the object that owns or created the childOwner + */ + public ChainedAllocatorOwner(AllocatorOwner childOwner, AllocatorOwner parentOwner) { + this.childOwner = childOwner; + this.parentOwner = parentOwner; + } + + @Override + public String toString() { + return childOwner + "(owned by " + parentOwner + ')'; + } + + @Override + public ExecutionControls getExecutionControls() { + return childOwner.getExecutionControls(); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java new file mode 100644 index 00000000000..8636d261c57 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java @@ -0,0 +1,47 @@ +/** + * 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.memory; + +/** + * Child allocator class. Only slightly different from the {@see RootAllocator}, + * in that these can't be created directly, but must be obtained from + * {@see BufferAllocator#newChildAllocator(AllocatorOwner, long, long, int)}. + + *

Child allocators can only be created by the root, or other children, so + * this class is package private.

+ */ +class ChildAllocator extends BaseAllocator { + /** + * Constructor. + * + * @param parentAllocator parent allocator -- the one creating this child + * @param allocatorOwner a handle to the object making the request + * @param allocationPolicy the allocation policy to use; the policy for all + * allocators must match for each invocation of a drillbit + * @param initReservation initial amount of space to reserve (obtained from the parent) + * @param maxAllocation maximum amount of space that can be obtained from this allocator; + * note this includes direct allocations (via {@see BufferAllocator#buffer(int, int)} + * et al) and requests from descendant allocators. Depending on the allocation policy in + * force, even less memory may be available + * @param flags one or more of BaseAllocator.F_* flags + */ + ChildAllocator(BaseAllocator parentAllocator, AllocatorOwner allocatorOwner, + AllocationPolicy allocationPolicy, long initReservation, long maxAllocation, int flags) { + super(parentAllocator, allocatorOwner, allocationPolicy, initReservation, maxAllocation, flags); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryException.java index 063f1c1d777..62fa68bfdbf 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryException.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryException.java @@ -20,32 +20,26 @@ import org.apache.drill.exec.exception.FragmentSetupException; public class OutOfMemoryException extends FragmentSetupException{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OutOfMemoryException.class); + private static final long serialVersionUID = -8089365467173775228L; +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OutOfMemoryException.class); public OutOfMemoryException() { - super(); - } - public OutOfMemoryException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + public OutOfMemoryException(String message, Throwable cause, boolean enableSuppression, + boolean writableStackTrace) { super(message, cause, enableSuppression, writableStackTrace); - } public OutOfMemoryException(String message, Throwable cause) { super(message, cause); - } public OutOfMemoryException(String message) { super(message); - } public OutOfMemoryException(Throwable cause) { super(cause); - } - - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryRuntimeException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryRuntimeException.java index 305eabd92bd..14cd48895b1 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryRuntimeException.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryRuntimeException.java @@ -19,31 +19,26 @@ import org.apache.drill.common.exceptions.DrillRuntimeException; -public class OutOfMemoryRuntimeException extends DrillRuntimeException{ +public class OutOfMemoryRuntimeException extends DrillRuntimeException { + private static final long serialVersionUID = 1241552417096596818L; public OutOfMemoryRuntimeException() { - super(); - } public OutOfMemoryRuntimeException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { super(message, cause, enableSuppression, writableStackTrace); - } public OutOfMemoryRuntimeException(String message, Throwable cause) { super(message, cause); - } public OutOfMemoryRuntimeException(String message) { super(message); - } public OutOfMemoryRuntimeException(Throwable cause) { super(cause); - } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocator.java new file mode 100644 index 00000000000..23557f9496b --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocator.java @@ -0,0 +1,119 @@ +/** + * 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.memory; + +import java.lang.management.ManagementFactory; + +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.PooledByteBufAllocatorL; + +import org.apache.drill.common.config.DrillConfig; +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.testing.ExecutionControls; + +import com.google.common.annotations.VisibleForTesting; + +/** + * The root allocator for using direct memory inside a Drillbit. Supports creating a + * tree of descendant child allocators. + */ +public class RootAllocator extends BaseAllocator { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootAllocator.class); + + // TODO these statics, and others in BaseAllocator, may be a problem for multiple in-process Drillbits + private static final PooledByteBufAllocatorL innerAllocator = PooledByteBufAllocatorL.DEFAULT; + private static long maxDirect; + + public static AllocationPolicy getAllocationPolicy() { + final String policyName = System.getProperty(ExecConstants.ALLOCATION_POLICY, + BaseAllocator.POLICY_LOCAL_MAX_NAME); // TODO try with PER_FRAGMENT_NAME + + switch(policyName) { + case POLICY_PER_FRAGMENT_NAME: + return POLICY_PER_FRAGMENT; + case POLICY_LOCAL_MAX_NAME: + return POLICY_LOCAL_MAX; + default: + throw new IllegalArgumentException("Unrecognized allocation policy name \"" + policyName + "\""); + } + } + + public RootAllocator(final DrillConfig drillConfig) { + this(getAllocationPolicy(), 0, Math.min( + DrillConfig.getMaxDirectMemory(), drillConfig.getLong(ExecConstants.TOP_LEVEL_MAX_ALLOC)), 0); + } + + public static long getMaxDirect() { + return maxDirect; + } + + /** + * Provide statistics via JMX for each RootAllocator. + */ + private class AllocatorsStats implements AllocatorsStatsMXBean { + @Override + public long getMaxDirectMemory() { + return maxDirect; + } + } + + private static class RootAllocatorOwner implements AllocatorOwner { + @Override + public ExecutionControls getExecutionControls() { + return null; + } + } + + @VisibleForTesting + public RootAllocator(final AllocationPolicy allocationPolicy, + final long initAllocation, final long maxReservation, final int flags) { + super(null, new RootAllocatorOwner(), allocationPolicy, initAllocation, maxDirect = maxReservation, flags); + assert (flags & F_LIMITING_ROOT) == 0 : "the RootAllocator shouldn't be a limiting root"; + + try { + final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); + final ObjectName objectName = new ObjectName("org.apache.drill.exec.memory:Allocators=" + id); + final AllocatorsStats mbean = new AllocatorsStats(); + mbs.registerMBean(mbean, objectName); + } catch(Exception e) { + logger.info("Exception setting up AllocatorsStatsMBean", e); + } + } + + @Override + public ByteBufAllocator getUnderlyingAllocator() { + return innerAllocator; + } + + @Override + protected boolean canIncreaseOwned(final long nBytes, final int flags) { + // the end total has already been checked against maxAllocation, so we can just return true + return true; + } + + /** + * Verify the accounting state of the allocation system. + */ + @VisibleForTesting + public void verify() { + verifyAllocator(); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorStatsMXBean.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorStatsMXBean.java new file mode 100644 index 00000000000..6d138997c14 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorStatsMXBean.java @@ -0,0 +1,47 @@ +/** + * 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.memory; + +/** + * JMX bean for getting statistics about allocators. + */ +public interface RootAllocatorStatsMXBean { + /** + * Get the amount of memory this allocator owns. This includes its + * allocated memory, and may include additional memory that it may hand + * out without going to its parent for more. + * + * @return the amount of memory owned by this allocator + */ + public long getOwnedMemory(); + + /** + * Get the amount of memory this allocator has allocated. This includes + * buffers it has allocated and memory it has given to its children to manage. + * + * @return the amount of memory allocated by this allocator + */ + public long getAllocatedMemory(); + + /** + * Get the number of child allocators this allocator owns. + * + * @return the number of child allocators this allocator owns + */ + public long getChildCount(); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java deleted file mode 100644 index 05849ea9ccc..00000000000 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java +++ /dev/null @@ -1,401 +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.memory; - -import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.DrillBuf; -import io.netty.buffer.PooledByteBufAllocatorL; -import io.netty.buffer.UnsafeDirectLittleEndian; - -import java.util.HashMap; -import java.util.IdentityHashMap; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.drill.common.config.DrillConfig; -import org.apache.drill.exec.ExecConstants; -import org.apache.drill.exec.ops.FragmentContext; -import org.apache.drill.exec.proto.ExecProtos.FragmentHandle; -import org.apache.drill.exec.testing.ControlsInjector; -import org.apache.drill.exec.testing.ControlsInjectorFactory; -import org.apache.drill.exec.util.AssertionUtil; -import org.apache.drill.exec.util.Pointer; - -public class TopLevelAllocator implements BufferAllocator { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TopLevelAllocator.class); - private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(TopLevelAllocator.class); - public static final String CHILD_BUFFER_INJECTION_SITE = "child.buffer"; - - public static long MAXIMUM_DIRECT_MEMORY; - - private static final boolean ENABLE_ACCOUNTING = AssertionUtil.isAssertionsEnabled(); - private final Map childrenMap; - private final PooledByteBufAllocatorL innerAllocator = PooledByteBufAllocatorL.DEFAULT; - private final Accountor acct; - private final boolean errorOnLeak; - private final DrillBuf empty; - private final DrillConfig config; - - @Deprecated - public TopLevelAllocator() { - this(DrillConfig.getMaxDirectMemory()); - } - - @Deprecated - public TopLevelAllocator(long maximumAllocation) { - this(null, maximumAllocation, true); - } - - private TopLevelAllocator(DrillConfig config, long maximumAllocation, boolean errorOnLeak){ - MAXIMUM_DIRECT_MEMORY = maximumAllocation; - this.config=(config!=null) ? config : DrillConfig.create(); - this.errorOnLeak = errorOnLeak; - this.acct = new Accountor(config, errorOnLeak, null, null, maximumAllocation, 0, true); - this.empty = DrillBuf.getEmpty(this, acct); - this.childrenMap = ENABLE_ACCOUNTING ? new IdentityHashMap() : null; - } - - public TopLevelAllocator(DrillConfig config) { - this(config, Math.min(DrillConfig.getMaxDirectMemory(), config.getLong(ExecConstants.TOP_LEVEL_MAX_ALLOC)), - config.getBoolean(ExecConstants.ERROR_ON_MEMORY_LEAK) - ); - } - - @Override - public boolean takeOwnership(DrillBuf buf) { - return buf.transferAccounting(acct); - } - - @Override - public boolean takeOwnership(DrillBuf buf, Pointer out) { - DrillBuf b = new DrillBuf(this, acct, buf); - out.value = b; - return acct.transferIn(b, b.capacity()); - } - - public DrillBuf buffer(int min, int max) { - if (min == 0) { - return empty; - } - if(!acct.reserve(min)) { - throw new OutOfMemoryRuntimeException(createErrorMsg(this, min)); - } - - try { - UnsafeDirectLittleEndian buffer = innerAllocator.directBuffer(min, max); - DrillBuf wrapped = new DrillBuf(this, acct, buffer); - acct.reserved(min, wrapped); - return wrapped; - } catch (OutOfMemoryError e) { - if ("Direct buffer memory".equals(e.getMessage())) { - acct.release(min); - throw new OutOfMemoryRuntimeException(createErrorMsg(this, min), e); - } else { - throw e; - } - } - } - - @Override - public DrillBuf buffer(int size) { - return buffer(size, size); - } - - @Override - public long getAllocatedMemory() { - return acct.getAllocation(); - } - - @Override - public long getPeakMemoryAllocation() { - return acct.getPeakMemoryAllocation(); - } - - @Override - public ByteBufAllocator getUnderlyingAllocator() { - return innerAllocator; - } - - @Override - public BufferAllocator getChildAllocator(FragmentContext context, long initialReservation, long maximumReservation, - boolean applyFragmentLimit) { - if(!acct.reserve(initialReservation)){ - logger.debug(String.format("You attempted to create a new child allocator with initial reservation %d but only %d bytes of memory were available.", initialReservation, acct.getCapacity() - acct.getAllocation())); - throw new OutOfMemoryRuntimeException( - String - .format( - "You attempted to create a new child allocator with initial reservation %d but only %d bytes of memory were available.", - initialReservation, acct.getCapacity() - acct.getAllocation())); - } - logger.debug("New child allocator with initial reservation {}", initialReservation); - ChildAllocator allocator = new ChildAllocator(context, acct, maximumReservation, initialReservation, childrenMap, applyFragmentLimit); - if(ENABLE_ACCOUNTING){ - childrenMap.put(allocator, Thread.currentThread().getStackTrace()); - } - - return allocator; - } - - @Override - public void resetFragmentLimits() { - acct.resetFragmentLimits(); - } - - @Override - public void setFragmentLimit(long limit){ - acct.setFragmentLimit(limit); - } - - @Override - public long getFragmentLimit(){ - return acct.getFragmentLimit(); - } - - @Override - public void close() { - if (ENABLE_ACCOUNTING) { - for (Entry child : childrenMap.entrySet()) { - if (!child.getKey().isClosed()) { - StringBuilder sb = new StringBuilder(); - StackTraceElement[] elements = child.getValue(); - for (int i = 0; i < elements.length; i++) { - sb.append("\t\t"); - sb.append(elements[i]); - sb.append("\n"); - } - throw new IllegalStateException("Failure while trying to close allocator: Child level allocators not closed. Stack trace: \n" + sb); - } - } - } - acct.close(); - } - - - - @Override - public DrillBuf getEmpty() { - return empty; - } - - - - private class ChildAllocator implements BufferAllocator { - private final DrillBuf empty; - private Accountor childAcct; - private Map children = new HashMap<>(); - private boolean closed = false; - private FragmentHandle handle; - private FragmentContext fragmentContext; - private Map thisMap; - private boolean applyFragmentLimit; - - public ChildAllocator(FragmentContext context, - Accountor parentAccountor, - long max, - long pre, - Map map, - boolean applyFragmentLimit) { - assert max >= pre; - this.applyFragmentLimit=applyFragmentLimit; - DrillConfig drillConf = context != null ? context.getConfig() : null; - childAcct = new Accountor(drillConf, errorOnLeak, context, parentAccountor, max, pre, applyFragmentLimit); - this.fragmentContext=context; - this.handle = context != null ? context.getHandle() : null; - thisMap = map; - this.empty = DrillBuf.getEmpty(this, childAcct); - } - - @Override - public boolean takeOwnership(DrillBuf buf) { - return buf.transferAccounting(childAcct); - } - - @Override - public boolean takeOwnership(DrillBuf buf, Pointer out) { - DrillBuf b = new DrillBuf(this, acct, buf); - out.value = b; - return acct.transferIn(b, b.capacity()); - } - - - @Override - public DrillBuf buffer(int size, int max) { - if (ENABLE_ACCOUNTING) { - injector.injectUnchecked(fragmentContext, CHILD_BUFFER_INJECTION_SITE); - } - - if (size == 0) { - return empty; - } - if(!childAcct.reserve(size)) { - throw new OutOfMemoryRuntimeException(createErrorMsg(this, size)); - } - - try { - UnsafeDirectLittleEndian buffer = innerAllocator.directBuffer(size, max); - DrillBuf wrapped = new DrillBuf(this, childAcct, buffer); - childAcct.reserved(buffer.capacity(), wrapped); - return wrapped; - } catch (OutOfMemoryError e) { - if ("Direct buffer memory".equals(e.getMessage())) { - childAcct.release(size); - throw new OutOfMemoryRuntimeException(createErrorMsg(this, size), e); - } else { - throw e; - } - } - } - - public DrillBuf buffer(int size) { - return buffer(size, size); - } - - @Override - public ByteBufAllocator getUnderlyingAllocator() { - return innerAllocator; - } - - @Override - public BufferAllocator getChildAllocator(FragmentContext context, long initialReservation, long maximumReservation, - boolean applyFragmentLimit) { - if (!childAcct.reserve(initialReservation)) { - throw new OutOfMemoryRuntimeException( - String - .format( - "You attempted to create a new child allocator with initial reservation %d but only %d bytes of memory were available.", - initialReservation, childAcct.getAvailable())); - } - logger.debug("New child allocator with initial reservation {}", initialReservation); - ChildAllocator newChildAllocator = new ChildAllocator(context, childAcct, maximumReservation, initialReservation, null, applyFragmentLimit); - this.children.put(newChildAllocator, Thread.currentThread().getStackTrace()); - return newChildAllocator; - } - - public PreAllocator getNewPreAllocator() { - return new PreAlloc(this, this.childAcct); - } - - @Override - public void resetFragmentLimits(){ - childAcct.resetFragmentLimits(); - } - - @Override - public void setFragmentLimit(long limit){ - childAcct.setFragmentLimit(limit); - } - - @Override - public long getFragmentLimit(){ - return childAcct.getFragmentLimit(); - } - - @Override - public void close() { - if (ENABLE_ACCOUNTING) { - if (thisMap != null) { - thisMap.remove(this); - } - for (ChildAllocator child : children.keySet()) { - if (!child.isClosed()) { - StringBuilder sb = new StringBuilder(); - StackTraceElement[] elements = children.get(child); - for (int i = 1; i < elements.length; i++) { - sb.append("\t\t"); - sb.append(elements[i]); - sb.append("\n"); - } - - - IllegalStateException e = new IllegalStateException(String.format( - "Failure while trying to close child allocator: Child level allocators not closed. Fragment %d:%d. Stack trace: \n %s", - handle.getMajorFragmentId(), handle.getMinorFragmentId(), sb.toString())); - if (errorOnLeak) { - throw e; - } else { - logger.warn("Memory leak.", e); - } - } - } - } - childAcct.close(); - closed = true; - } - - public boolean isClosed() { - return closed; - } - - @Override - public long getAllocatedMemory() { - return childAcct.getAllocation(); - } - - @Override - public long getPeakMemoryAllocation() { - return childAcct.getPeakMemoryAllocation(); - } - - @Override - public DrillBuf getEmpty() { - return empty; - } - - - } - - public PreAllocator getNewPreAllocator() { - return new PreAlloc(this, this.acct); - } - - public class PreAlloc implements PreAllocator{ - int bytes = 0; - final Accountor acct; - final BufferAllocator allocator; - private PreAlloc(BufferAllocator allocator, Accountor acct) { - this.acct = acct; - this.allocator = allocator; - } - - /** - * - */ - public boolean preAllocate(int bytes) { - - if (!acct.reserve(bytes)) { - return false; - } - this.bytes += bytes; - return true; - - } - - - public DrillBuf getAllocation() { - DrillBuf b = new DrillBuf(allocator, acct, innerAllocator.directBuffer(bytes, bytes)); - acct.reserved(bytes, b); - return b; - } - } - - private static String createErrorMsg(final BufferAllocator allocator, final int size) { - return String.format("Unable to allocate buffer of size %d due to memory limit. Current allocation: %d", - size, allocator.getAllocatedMemory()); - } -} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BufferManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BufferManager.java index c953bb376d1..4f87575d624 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BufferManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BufferManager.java @@ -21,8 +21,6 @@ import io.netty.buffer.DrillBuf; import org.apache.drill.exec.memory.BufferAllocator; -import java.io.Closeable; - /** * Manages a list of {@link DrillBuf}s that can be reallocated as needed. Upon * re-allocation the old buffer will be freed. Managing a list of these buffers @@ -38,7 +36,6 @@ * and {@link QueryContext}. */ public class BufferManager implements AutoCloseable { - private LongObjectOpenHashMap managedBuffers = new LongObjectOpenHashMap<>(); private final BufferAllocator allocator; @@ -51,11 +48,12 @@ public BufferManager(BufferAllocator allocator, FragmentContext fragmentContext) this.fragmentContext = fragmentContext; } + @Override public void close() throws Exception { Object[] mbuffers = ((LongObjectOpenHashMap)(Object)managedBuffers).values; for (int i =0; i < mbuffers.length; i++) { if (managedBuffers.allocated[i]) { - ((DrillBuf)mbuffers[i]).release(); + ((DrillBuf)mbuffers[i]).release(1); } } managedBuffers.clear(); @@ -65,7 +63,7 @@ public DrillBuf replace(DrillBuf old, int newSize) { if (managedBuffers.remove(old.memoryAddress()) == null) { throw new IllegalStateException("Tried to remove unmanaged buffer."); } - old.release(); + old.release(1); return getManagedBuffer(newSize); } @@ -80,5 +78,4 @@ public DrillBuf getManagedBuffer(int size) { newBuf.setBufferManager(this); return newBuf; } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java index d5e85eeff90..cac972d4fb3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java @@ -24,7 +24,7 @@ import java.util.Map; import org.apache.calcite.schema.SchemaPlus; - +import org.apache.drill.common.StackTrace; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.exceptions.ExecutionSetupException; import org.apache.drill.common.exceptions.UserException; @@ -33,7 +33,9 @@ import org.apache.drill.exec.expr.ClassGenerator; import org.apache.drill.exec.expr.CodeGenerator; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; +import org.apache.drill.exec.memory.AllocatorOwner; import org.apache.drill.exec.memory.BufferAllocator; +import org.apache.drill.exec.memory.ChainedAllocatorOwner; import org.apache.drill.exec.memory.OutOfMemoryException; import org.apache.drill.exec.physical.base.PhysicalOperator; import org.apache.drill.exec.memory.OutOfMemoryRuntimeException; @@ -80,7 +82,7 @@ public class FragmentContext implements AutoCloseable, UdfUtilities { private final ContextInformation contextInformation; private IncomingBuffers buffers; private final OptionManager fragmentOptions; - private final BufferManager bufferManager; + private final BufferManager bufferManager; // TODO eliminate in favor of using BufferAllocator.TRACK_BUFFERS private ExecutorState executorState; private final ExecutionControls executionControls; @@ -103,6 +105,18 @@ public void interrupt(final InterruptedException e) { private final RpcOutcomeListener statusHandler = new StatusHandler(exceptionConsumer, sendingAccountor); private final AccountingUserConnection accountingUserConnection; + private final AllocatorOwner allocatorOwner = new AllocatorOwner() { + @Override + public String toString() { + return fragment.getHandle().toString(); + } + + @Override + public ExecutionControls getExecutionControls() { + return FragmentContext.this.getExecutionControls(); + } + }; + /** * Create a FragmentContext instance for non-root fragment. * @@ -110,6 +124,7 @@ public void interrupt(final InterruptedException e) { * @param fragment Fragment implementation. * @param funcRegistry FunctionImplementationRegistry. * @throws ExecutionSetupException + * */ public FragmentContext(final DrillbitContext dbContext, final PlanFragment fragment, final FunctionImplementationRegistry funcRegistry) throws ExecutionSetupException { @@ -157,9 +172,10 @@ public FragmentContext(final DrillbitContext dbContext, final PlanFragment fragm // Add the fragment context to the root allocator. // The QueryManager will call the root allocator to recalculate all the memory limits for all the fragments try { - allocator = context.getAllocator().getChildAllocator(this, fragment.getMemInitial(), fragment.getMemMax(), true); - Preconditions.checkNotNull(allocator, "Unable to acuqire allocator"); - } catch(final OutOfMemoryException | OutOfMemoryRuntimeException e) { + allocator = context.getAllocator().newChildAllocator( + allocatorOwner, fragment.getMemInitial(), fragment.getMemMax(), BufferAllocator.F_LIMITING_ROOT); + Preconditions.checkNotNull(allocator, "Unable to acquire allocator"); + } catch(final OutOfMemoryRuntimeException e) { throw UserException.memoryError(e) .addContext("Fragment", getHandle().getMajorFragmentId() + ":" + getHandle().getMinorFragmentId()) .build(logger); @@ -275,7 +291,6 @@ private String getFragIdString() { * Get this fragment's allocator. * @return the allocator */ - @Deprecated public BufferAllocator getAllocator() { if (allocator == null) { logger.debug("Fragment: " + getFragIdString() + " Allocator is NULL"); @@ -283,10 +298,16 @@ public BufferAllocator getAllocator() { return allocator; } - public BufferAllocator getNewChildAllocator(final long initialReservation, + public BufferAllocator newChildAllocator(final AllocatorOwner allocatorOwner, final long initialReservation, final long maximumReservation, - final boolean applyFragmentLimit) throws OutOfMemoryException { - return allocator.getChildAllocator(this, initialReservation, maximumReservation, applyFragmentLimit); + final boolean applyFragmentLimit) { +/* TODO(cwestin) Do we need to handle these? + if (!applyFragmentLimit) { + throw new IllegalArgumentException("applyFragmentLimit is false"); + } +*/ + return allocator.newChildAllocator(new ChainedAllocatorOwner(allocatorOwner, this.allocatorOwner), + initialReservation, maximumReservation, 0); } public T getImplementationClass(final ClassGenerator cg) @@ -333,6 +354,10 @@ public OperatorContext newOperatorContext(PhysicalOperator popConfig, OperatorSt throws OutOfMemoryException { OperatorContextImpl context = new OperatorContextImpl(popConfig, this, stats, applyFragmentLimit); contexts.add(context); + if (logger.isDebugEnabled()) { + logger.debug("created OperatorContextImpl {} at\n{}", + System.identityHashCode(context), new StackTrace()); + } return context; } @@ -340,6 +365,10 @@ public OperatorContext newOperatorContext(PhysicalOperator popConfig, boolean ap throws OutOfMemoryException { OperatorContextImpl context = new OperatorContextImpl(popConfig, this, applyFragmentLimit); contexts.add(context); + if (logger.isDebugEnabled()) { + logger.debug("created OperatorContextImpl {} at\n{}", + System.identityHashCode(context), new StackTrace()); + } return context; } @@ -363,6 +392,7 @@ public DrillConfig getConfig() { return context.getConfig(); } + @Deprecated public void setFragmentLimit(final long limit) { allocator.setFragmentLimit(limit); } @@ -392,6 +422,10 @@ public void close() { // close operator context for (OperatorContextImpl opContext : contexts) { suppressingClose(opContext); + if (logger.isDebugEnabled()) { + logger.debug(String.format("closed OperatorContextImpl %d", + System.identityHashCode(opContext))); + } } suppressingClose(bufferManager); @@ -459,7 +493,5 @@ public interface ExecutorState { @VisibleForTesting @Deprecated public Throwable getFailureCause(); - } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java index 7eb7d8aceca..5672c4934a2 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java @@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration; public abstract class OperatorContext { - public abstract DrillBuf replace(DrillBuf old, int newSize); public abstract DrillBuf getManagedBuffer(); @@ -57,5 +56,4 @@ public static int getChildCount(PhysicalOperator popConfig) { } return i; } - -} \ No newline at end of file +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java index ce9f351fa1e..6e7c3dc7ba2 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java @@ -18,14 +18,17 @@ package org.apache.drill.exec.ops; import com.google.common.base.Preconditions; + import io.netty.buffer.DrillBuf; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.exceptions.DrillRuntimeException; +import org.apache.drill.exec.memory.AllocatorOwner; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.OutOfMemoryException; import org.apache.drill.exec.physical.base.PhysicalOperator; import com.carrotsearch.hppc.LongObjectOpenHashMap; + import org.apache.drill.exec.testing.ExecutionControls; import org.apache.drill.exec.store.dfs.DrillFileSystem; import org.apache.hadoop.conf.Configuration; @@ -33,20 +36,31 @@ import java.io.IOException; class OperatorContextImpl extends OperatorContext implements AutoCloseable { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorContextImpl.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorContextImpl.class); private final BufferAllocator allocator; private final ExecutionControls executionControls; private boolean closed = false; - private PhysicalOperator popConfig; - private OperatorStats stats; - private LongObjectOpenHashMap managedBuffers = new LongObjectOpenHashMap<>(); - private final boolean applyFragmentLimit; + private final PhysicalOperator popConfig; + private final OperatorStats stats; + private final LongObjectOpenHashMap managedBuffers = new LongObjectOpenHashMap<>(); private DrillFileSystem fs; - public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, boolean applyFragmentLimit) throws OutOfMemoryException { - this.applyFragmentLimit=applyFragmentLimit; - this.allocator = context.getNewChildAllocator(popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit); + private final AllocatorOwner allocatorOwner = new AllocatorOwner() { + @Override + public ExecutionControls getExecutionControls() { + return executionControls; + } + + @Override + public String toString() { + return String.format("OperatorContextImpl %s", System.identityHashCode(OperatorContextImpl.this)); + } + }; + + public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, boolean applyFragmentLimit) { + this.allocator = context.newChildAllocator(allocatorOwner, + popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit); this.popConfig = popConfig; OpProfileDef def = new OpProfileDef(popConfig.getOperatorId(), popConfig.getOperatorType(), getChildCount(popConfig)); @@ -54,14 +68,15 @@ public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, executionControls = context.getExecutionControls(); } - public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, OperatorStats stats, boolean applyFragmentLimit) throws OutOfMemoryException { - this.applyFragmentLimit=applyFragmentLimit; - this.allocator = context.getNewChildAllocator(popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit); + public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, OperatorStats stats, boolean applyFragmentLimit) { + this.allocator = context.newChildAllocator(allocatorOwner, + popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit); this.popConfig = popConfig; this.stats = stats; executionControls = context.getExecutionControls(); } + @Override public DrillBuf replace(DrillBuf old, int newSize) { if (managedBuffers.remove(old.memoryAddress()) == null) { throw new IllegalStateException("Tried to remove unmanaged buffer."); @@ -70,10 +85,12 @@ public DrillBuf replace(DrillBuf old, int newSize) { return getManagedBuffer(newSize); } + @Override public DrillBuf getManagedBuffer() { return getManagedBuffer(256); } + @Override public DrillBuf getManagedBuffer(int size) { DrillBuf newBuf = allocator.buffer(size); managedBuffers.put(newBuf.memoryAddress(), newBuf); @@ -81,10 +98,12 @@ public DrillBuf getManagedBuffer(int size) { return newBuf; } + @Override public ExecutionControls getExecutionControls() { return executionControls; } + @Override public BufferAllocator getAllocator() { if (allocator == null) { throw new UnsupportedOperationException("Operator context does not have an allocator"); @@ -99,10 +118,14 @@ public boolean isClosed() { @Override public void close() { if (closed) { - logger.debug("Attempted to close Operator context for {}, but context is already closed", popConfig != null ? popConfig.getClass().getName() : null); + logger.debug("Attempted to close Operator context for {}, but context is already closed", + popConfig != null ? popConfig.getClass().getName() : null); return; } - logger.debug("Closing context for {}", popConfig != null ? popConfig.getClass().getName() : null); + logger.debug("Closing context for {}, allocatorOwner {}, allocator[{}]", + popConfig != null ? popConfig.getClass().getName() : null, + allocatorOwner, + allocator != null ? allocator.getId() : ""); // release managed buffers. Object[] buffers = ((LongObjectOpenHashMap)(Object)managedBuffers).values; @@ -113,7 +136,7 @@ public void close() { } if (allocator != null) { - allocator.close(); + DrillAutoCloseables.closeNoChecked(allocator); } if (fs != null) { @@ -126,6 +149,7 @@ public void close() { closed = true; } + @Override public OperatorStats getStats() { return stats; } @@ -136,5 +160,4 @@ public DrillFileSystem newFileSystem(Configuration conf) throws IOException { fs = new DrillFileSystem(conf, getStats()); return fs; } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java index 60fba0f2d0b..6780f9df7f7 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java @@ -93,12 +93,9 @@ public QueryContext(final UserSession session, final DrillbitContext drillbitCon queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaName()); contextInformation = new ContextInformation(session.getCredentials(), queryContextInfo); - try { - allocator = drillbitContext.getAllocator().getChildAllocator(null, INITIAL_OFF_HEAP_ALLOCATION_IN_BYTES, + allocator = drillbitContext.getAllocator().getChildAllocator(null, INITIAL_OFF_HEAP_ALLOCATION_IN_BYTES, MAX_OFF_HEAP_ALLOCATION_IN_BYTES, false); - } catch (OutOfMemoryException e) { - throw new DrillRuntimeException("Error creating off-heap allocator for planning context.",e); - } + // TODO(DRILL-1942) the new allocator has this capability built-in, so this can be removed once that is available bufferManager = new BufferManager(this.allocator, null); viewExpansionContext = new ViewExpansionContext(this); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java index accce43ed55..9301650d0ec 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java @@ -19,6 +19,7 @@ import java.util.List; +import org.apache.drill.common.DeferredException; import org.apache.drill.exec.memory.OutOfMemoryException; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.ops.OpProfileDef; @@ -30,6 +31,8 @@ import org.apache.drill.exec.record.RecordBatch; import org.apache.drill.exec.record.RecordBatch.IterOutcome; +import com.google.common.base.Supplier; + public abstract class BaseRootExec implements RootExec { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseRootExec.class); @@ -59,6 +62,24 @@ public BaseRootExec(final FragmentContext fragmentContext, final OperatorContext void setOperators(List operators) { this.operators = operators; + + if (logger.isDebugEnabled()) { + final StringBuilder sb = new StringBuilder(); + sb.append("BaseRootExec("); + sb.append(Integer.toString(System.identityHashCode(this))); + sb.append(") operators: "); + for(final CloseableRecordBatch crb : operators) { + sb.append(crb.getClass().getName()); + sb.append(' '); + sb.append(Integer.toString(System.identityHashCode(crb))); + sb.append(", "); + } + + // Cut off the last trailing comma and space + sb.setLength(sb.length() - 2); + + logger.debug(sb.toString()); + } } @Override @@ -117,13 +138,25 @@ public void close() throws Exception { // close all operators. if (operators != null) { - for (CloseableRecordBatch b : operators) { - try { - b.close(); - } catch (Exception e) { - fragmentContext.fail(e); + final DeferredException df = new DeferredException(new Supplier() { + @Override + public Exception get() { + return new RuntimeException("Error closing operators"); + } + }); + + for (final CloseableRecordBatch crb : operators) { + df.suppressingClose(crb); + if (logger.isDebugEnabled()) { + logger.debug(String.format("closed operator %d", System.identityHashCode(crb))); } } + + try { + df.close(); + } catch (Exception e) { + fragmentContext.fail(e); + } } } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java index 66558befe04..0244fa261ea 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java @@ -17,7 +17,6 @@ */ package org.apache.drill.exec.physical.impl; -import java.io.Closeable; import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.LinkedList; @@ -44,10 +43,10 @@ * Create RecordBatch tree (PhysicalOperator implementations) for a given PhysicalOperator tree. */ public class ImplCreator { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImplCreator.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImplCreator.class); private RootExec root = null; - private LinkedList operators = Lists.newLinkedList(); + private final LinkedList operators = Lists.newLinkedList(); private ImplCreator() {} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java index 6bf1280ae09..79906ad16c7 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java @@ -85,13 +85,14 @@ public class ScanBatch implements CloseableRecordBatch { private SchemaChangeCallBack callBack = new SchemaChangeCallBack(); public ScanBatch(PhysicalOperator subScanConfig, FragmentContext context, OperatorContext oContext, - Iterator readers, List partitionColumns, List selectedPartitionColumns) throws ExecutionSetupException { + Iterator readers, List partitionColumns, List selectedPartitionColumns) + throws ExecutionSetupException { this.context = context; this.readers = readers; if (!readers.hasNext()) { throw new ExecutionSetupException("A scan batch must contain at least one reader."); } - this.currentReader = readers.next(); + currentReader = readers.next(); this.oContext = oContext; boolean setup = false; @@ -102,36 +103,38 @@ public ScanBatch(PhysicalOperator subScanConfig, FragmentContext context, Operat } finally { // if we had an exception during setup, make sure to release existing data. if (!setup) { - currentReader.cleanup(); + try { + currentReader.close(); + } catch(final Exception e) { + throw new ExecutionSetupException(e); + } } oContext.getStats().stopProcessing(); } this.partitionColumns = partitionColumns.iterator(); - this.partitionValues = this.partitionColumns.hasNext() ? this.partitionColumns.next() : null; + partitionValues = this.partitionColumns.hasNext() ? this.partitionColumns.next() : null; this.selectedPartitionColumns = selectedPartitionColumns; // TODO Remove null check after DRILL-2097 is resolved. That JIRA refers to test cases that do not initialize // options; so labelValue = null. final OptionValue labelValue = context.getOptions().getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL); - this.partitionColumnDesignator = labelValue == null ? "dir" : labelValue.string_val; + partitionColumnDesignator = labelValue == null ? "dir" : labelValue.string_val; addPartitionVectors(); } - public ScanBatch(PhysicalOperator subScanConfig, FragmentContext context, Iterator readers) throws ExecutionSetupException { + public ScanBatch(PhysicalOperator subScanConfig, FragmentContext context, Iterator readers) + throws ExecutionSetupException { this(subScanConfig, context, context.newOperatorContext(subScanConfig, false /* ScanBatch is not subject to fragment memory limit */), readers, Collections. emptyList(), Collections. emptyList()); } + @Override public FragmentContext getContext() { return context; } - public OperatorContext getOperatorContext() { - return oContext; - } - @Override public BatchSchema getSchema() { return schema; @@ -155,6 +158,12 @@ private void releaseAssets() { container.zeroVectors(); } + private void clearFieldVectorMap() { + for (final ValueVector v : fieldVectorMap.values()) { + v.clear(); + } + } + @Override public IterOutcome next() { if (done) { @@ -168,15 +177,13 @@ public IterOutcome next() { currentReader.allocate(fieldVectorMap); } catch (OutOfMemoryException | OutOfMemoryRuntimeException e) { logger.debug("Caught Out of Memory Exception", e); - for (ValueVector v : fieldVectorMap.values()) { - v.clear(); - } + clearFieldVectorMap(); return IterOutcome.OUT_OF_MEMORY; } while ((recordCount = currentReader.next()) == 0) { try { if (!readers.hasNext()) { - currentReader.cleanup(); + currentReader.close(); releaseAssets(); done = true; if (mutator.isNewSchema()) { @@ -186,7 +193,7 @@ public IterOutcome next() { return IterOutcome.NONE; } - currentReader.cleanup(); + currentReader.close(); currentReader = readers.next(); partitionValues = partitionColumns.hasNext() ? partitionColumns.next() : null; currentReader.setup(oContext, mutator); @@ -194,9 +201,7 @@ public IterOutcome next() { currentReader.allocate(fieldVectorMap); } catch (OutOfMemoryException e) { logger.debug("Caught OutOfMemoryException"); - for (ValueVector v : fieldVectorMap.values()) { - v.clear(); - } + clearFieldVectorMap(); return IterOutcome.OUT_OF_MEMORY; } addPartitionVectors(); @@ -233,7 +238,7 @@ public IterOutcome next() { } } - private void addPartitionVectors() throws ExecutionSetupException{ + private void addPartitionVectors() throws ExecutionSetupException { try { if (partitionVectors != null) { for (ValueVector v : partitionVectors) { @@ -242,8 +247,10 @@ private void addPartitionVectors() throws ExecutionSetupException{ } partitionVectors = Lists.newArrayList(); for (int i : selectedPartitionColumns) { - MaterializedField field = MaterializedField.create(SchemaPath.getSimplePath(partitionColumnDesignator + i), Types.optional(MinorType.VARCHAR)); - ValueVector v = mutator.addField(field, NullableVarCharVector.class); + final MaterializedField field = + MaterializedField.create(SchemaPath.getSimplePath(partitionColumnDesignator + i), + Types.optional(MinorType.VARCHAR)); + final ValueVector v = mutator.addField(field, NullableVarCharVector.class); partitionVectors.add(v); } } catch(SchemaChangeException e) { @@ -253,12 +260,12 @@ private void addPartitionVectors() throws ExecutionSetupException{ private void populatePartitionVectors() { for (int index = 0; index < selectedPartitionColumns.size(); index++) { - int i = selectedPartitionColumns.get(index); - NullableVarCharVector v = (NullableVarCharVector) partitionVectors.get(index); + final int i = selectedPartitionColumns.get(index); + final NullableVarCharVector v = (NullableVarCharVector) partitionVectors.get(index); if (partitionValues.length > i) { - String val = partitionValues[i]; + final String val = partitionValues[i]; AllocationHelper.allocate(v, recordCount, val.length()); - byte[] bytes = val.getBytes(); + final byte[] bytes = val.getBytes(); for (int j = 0; j < recordCount; j++) { v.getMutator().setSafe(j, bytes, 0, bytes.length); } @@ -290,27 +297,24 @@ public VectorWrapper getValueAccessorById(Class clazz, int... ids) { return container.getValueAccessorById(clazz, ids); } - - private class Mutator implements OutputMutator { + private boolean schemaChange = true; - boolean schemaChange = true; - - @SuppressWarnings("unchecked") @Override public T addField(MaterializedField field, Class clazz) throws SchemaChangeException { // Check if the field exists ValueVector v = fieldVectorMap.get(field.key()); - if (v == null || v.getClass() != clazz) { // Field does not exist add it to the map and the output container v = TypeHelper.getNewVector(field, oContext.getAllocator(), callBack); if (!clazz.isAssignableFrom(v.getClass())) { - throw new SchemaChangeException(String.format("The class that was provided %s does not correspond to the expected vector type of %s.", clazz.getSimpleName(), v.getClass().getSimpleName())); + throw new SchemaChangeException(String.format( + "The class that was provided %s does not correspond to the expected vector type of %s.", + clazz.getSimpleName(), v.getClass().getSimpleName())); } - ValueVector old = fieldVectorMap.put(field.key(), v); - if(old != null){ + final ValueVector old = fieldVectorMap.put(field.key(), v); + if (old != null) { old.clear(); container.remove(old); } @@ -320,12 +324,12 @@ public T addField(MaterializedField field, Class claz schemaChange = true; } - return (T) v; + return clazz.cast(v); } @Override public void allocate(int recordCount) { - for (ValueVector v : fieldVectorMap.values()) { + for (final ValueVector v : fieldVectorMap.values()) { AllocationHelper.allocate(v, recordCount, 50, 10); } } @@ -357,18 +361,17 @@ public WritableBatch getWritableBatch() { } @Override - public void close() { + public void close() throws Exception { container.clear(); - for (ValueVector v : partitionVectors) { + for (final ValueVector v : partitionVectors) { v.clear(); } fieldVectorMap.clear(); - currentReader.cleanup(); + currentReader.close(); } @Override public VectorContainer getOutgoingContainer() { throw new UnsupportedOperationException(String.format(" You should not call getOutgoingContainer() for class %s", this.getClass().getCanonicalName())); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java index 363205c0b9d..9f4f15ab880 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java @@ -43,14 +43,15 @@ public class ScreenCreator implements RootCreator{ private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(ScreenCreator.class); @Override - public RootExec getRoot(FragmentContext context, Screen config, List children) throws ExecutionSetupException { + public RootExec getRoot(FragmentContext context, Screen config, List children) + throws ExecutionSetupException { Preconditions.checkNotNull(children); Preconditions.checkArgument(children.size() == 1); return new ScreenRoot(context, children.iterator().next(), config); } - static class ScreenRoot extends BaseRootExec { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenRoot.class); private final RecordBatch incoming; private final FragmentContext context; private final AccountingUserConnection userConnection; @@ -71,7 +72,7 @@ public ScreenRoot(FragmentContext context, RecordBatch incoming, Screen config) super(context, config); this.context = context; this.incoming = incoming; - this.userConnection = context.getUserDataTunnel(); + userConnection = context.getUserDataTunnel(); } @Override @@ -87,10 +88,10 @@ public boolean innerNext() { if (firstBatch) { // this is the only data message sent to the client and may contain the schema QueryWritableBatch batch; - QueryData header = QueryData.newBuilder() // - .setQueryId(context.getHandle().getQueryId()) // - .setRowCount(0) // - .setDef(RecordBatchDef.getDefaultInstance()) // + QueryData header = QueryData.newBuilder() + .setQueryId(context.getHandle().getQueryId()) + .setRowCount(0) + .setDef(RecordBatchDef.getDefaultInstance()) .build(); batch = new QueryWritableBatch(header); @@ -109,7 +110,7 @@ public boolean innerNext() { //$FALL-THROUGH$ case OK: injector.injectPause(context.getExecutionControls(), "sending-data", logger); - QueryWritableBatch batch = materializer.convertNext(); + final QueryWritableBatch batch = materializer.convertNext(); updateStats(batch); stats.startWait(); try { @@ -133,13 +134,10 @@ RecordBatch getIncoming() { return incoming; } - @Override public void close() throws Exception { injector.injectPause(context.getExecutionControls(), "send-complete", logger); super.close(); } } - - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java index f38275910a7..dc9dbbc2154 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java @@ -19,6 +19,7 @@ import java.util.List; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.exceptions.ExecutionSetupException; import org.apache.drill.exec.memory.OutOfMemoryException; import org.apache.drill.exec.memory.OutOfMemoryRuntimeException; @@ -52,7 +53,6 @@ public static class SingleSenderRootExec extends BaseRootExec { private RecordBatch incoming; private AccountingDataTunnel tunnel; private FragmentHandle handle; - private SingleSender config; private int recMajor; private volatile boolean ok = true; private volatile boolean done = false; @@ -69,11 +69,10 @@ public int metricId() { public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config) throws OutOfMemoryException { super(context, context.newOperatorContext(config, null, false), config); this.incoming = batch; - assert(incoming != null); - this.handle = context.getHandle(); - this.config = config; - this.recMajor = config.getOppositeMajorFragmentId(); - this.tunnel = context.getDataTunnel(config.getDestination()); + assert incoming != null; + handle = context.getHandle(); + recMajor = config.getOppositeMajorFragmentId(); + tunnel = context.getDataTunnel(config.getDestination()); oppositeHandle = handle.toBuilder() .setMajorFragmentId(config.getOppositeMajorFragmentId()) .setMinorFragmentId(config.getOppositeMinorFragmentId()) @@ -104,10 +103,10 @@ public boolean innerNext() { case STOP: case NONE: // if we didn't do anything yet, send an empty schema. - final BatchSchema sendSchema = incoming.getSchema() == null ? BatchSchema.newBuilder().build() : incoming - .getSchema(); + final BatchSchema sendSchema = incoming.getSchema() == null ? + BatchSchema.newBuilder().build() : incoming.getSchema(); - FragmentWritableBatch b2 = FragmentWritableBatch.getEmptyLastWithSchema(handle.getQueryId(), + final FragmentWritableBatch b2 = FragmentWritableBatch.getEmptyLastWithSchema(handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, oppositeHandle.getMinorFragmentId(), sendSchema); stats.startWait(); @@ -120,8 +119,10 @@ public boolean innerNext() { case OK_NEW_SCHEMA: case OK: - FragmentWritableBatch batch = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), - handle.getMinorFragmentId(), recMajor, oppositeHandle.getMinorFragmentId(), incoming.getWritableBatch()); + final FragmentWritableBatch batch = new FragmentWritableBatch( + false, handle.getQueryId(), handle.getMajorFragmentId(), + handle.getMinorFragmentId(), recMajor, oppositeHandle.getMinorFragmentId(), + incoming.getWritableBatch()); updateStats(batch); stats.startWait(); try { @@ -146,5 +147,4 @@ public void receivingFragmentFinished(FragmentHandle handle) { done = true; } } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java index 7e22e659a13..5cdfc5d37df 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java @@ -38,10 +38,10 @@ import com.google.common.base.Stopwatch; public abstract class PriorityQueueTemplate implements PriorityQueue { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueTemplate.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueTemplate.class); - private SelectionVector4 heapSv4;//This holds the heap - private SelectionVector4 finalSv4;//This is for final sorted output + private SelectionVector4 heapSv4; //This holds the heap + private SelectionVector4 finalSv4; //This is for final sorted output private ExpandableHyperContainer hyperBatch; private FragmentContext context; private BufferAllocator allocator; @@ -60,6 +60,7 @@ public void init(int limit, FragmentContext context, BufferAllocator allocator, this.hasSv2 = hasSv2; } + @Override public void resetQueue(VectorContainer container, SelectionVector4 v4) throws SchemaChangeException { assert container.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.FOUR_BYTE; BatchSchema schema = container.getSchema(); @@ -69,10 +70,10 @@ public void resetQueue(VectorContainer container, SelectionVector4 v4) throws Sc newContainer.add(container.getValueAccessorById(field.getValueClass(), ids).getValueVectors()); } newContainer.buildSchema(BatchSchema.SelectionVectorMode.FOUR_BYTE); - this.hyperBatch = new ExpandableHyperContainer(newContainer); - this.batchCount = hyperBatch.iterator().next().getValueVectors().length; + hyperBatch = new ExpandableHyperContainer(newContainer); + batchCount = hyperBatch.iterator().next().getValueVectors().length; final DrillBuf drillBuf = allocator.buffer(4 * (limit + 1)); - this.heapSv4 = new SelectionVector4(drillBuf, limit, Character.MAX_VALUE); + heapSv4 = new SelectionVector4(drillBuf, limit, Character.MAX_VALUE); for (int i = 0; i < v4.getTotalCount(); i++) { heapSv4.set(i, v4.get(i)); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java index 516b0282fb5..9e243c856d3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.expression.ErrorCollector; import org.apache.drill.common.expression.ErrorCollectorImpl; import org.apache.drill.common.expression.LogicalExpression; @@ -65,7 +66,7 @@ import com.sun.codemodel.JExpr; public class TopNBatch extends AbstractRecordBatch { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TopNBatch.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TopNBatch.class); private static final long MAX_SORT_BYTES = 1L * 1024 * 1024 * 1024; private final int batchPurgeThreshold; @@ -109,7 +110,7 @@ public SelectionVector4 getSelectionVector4() { } @Override - public void close() { + public void close() throws Exception { if (sv4 != null) { sv4.clear(); } @@ -122,20 +123,20 @@ public void close() { @Override public void buildSchema() throws SchemaChangeException { VectorContainer c = new VectorContainer(oContext); - IterOutcome outcome = next(incoming); + final IterOutcome outcome = next(incoming); switch (outcome) { case OK: case OK_NEW_SCHEMA: - for (VectorWrapper w : incoming) { - ValueVector v = c.addOrGet(w.getField()); + for (final VectorWrapper w : incoming) { + final ValueVector v = c.addOrGet(w.getField()); if (v instanceof AbstractContainerVector) { w.getValueVector().makeTransferPair(v); v.clear(); } } c = VectorContainer.canonicalize(c); - for (VectorWrapper w : c) { - ValueVector v = container.addOrGet(w.getField()); + for (final VectorWrapper w : c) { + final ValueVector v = container.addOrGet(w.getField()); if (v instanceof AbstractContainerVector) { w.getValueVector().makeTransferPair(v); v.clear(); @@ -209,7 +210,7 @@ public IterOutcome innerNext() { // fall through. case OK: if (incoming.getRecordCount() == 0) { - for (VectorWrapper w : incoming) { + for (VectorWrapper w : incoming) { w.clear(); } break; @@ -251,7 +252,7 @@ public IterOutcome innerNext() { this.sv4 = priorityQueue.getFinalSv4(); container.clear(); - for (VectorWrapper w : priorityQueue.getHyperBatch()) { + for (VectorWrapper w : priorityQueue.getHyperBatch()) { container.add(w.getValueVectors()); } container.buildSchema(BatchSchema.SelectionVectorMode.FOUR_BYTE); @@ -280,12 +281,12 @@ private void purge() throws SchemaChangeException { } else { for (VectorWrapper i : batch) { - ValueVector v = TypeHelper.getNewVector(i.getField(), oContext.getAllocator()); + final ValueVector v = TypeHelper.getNewVector(i.getField(), oContext.getAllocator()); newContainer.add(v); } copier.setupRemover(context, batch, newBatch); } - SortRecordBatchBuilder builder = new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES); + final SortRecordBatchBuilder builder = new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES); try { do { int count = selectionVector4.getCount(); @@ -308,7 +309,7 @@ private void purge() throws SchemaChangeException { builder.getSv4().clear(); selectionVector4.clear(); } finally { - builder.close(); + DrillAutoCloseables.closeNoChecked(builder); } logger.debug("Took {} us to purge", watch.elapsed(TimeUnit.MICROSECONDS)); } @@ -366,12 +367,10 @@ protected void killIncoming(boolean sendUpstream) { incoming.kill(sendUpstream); } - public static class SimpleRecordBatch implements RecordBatch { - - private VectorContainer container; - private SelectionVector4 sv4; - private FragmentContext context; + private final VectorContainer container; + private final SelectionVector4 sv4; + private final FragmentContext context; public SimpleRecordBatch(VectorContainer container, SelectionVector4 sv4, FragmentContext context) { this.container = container; @@ -379,6 +378,10 @@ public SimpleRecordBatch(VectorContainer container, SelectionVector4 sv4, Fragme this.context = context; } + @Override + public void close() throws Exception { + } + @Override public FragmentContext getContext() { return context; @@ -441,7 +444,5 @@ public Iterator> iterator() { public VectorContainer getOutgoingContainer() { throw new UnsupportedOperationException(String.format(" You should not call getOutgoingContainer() for class %s", this.getClass().getCanonicalName())); } - } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java index 5fe76672ba7..81d4c355582 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java @@ -49,14 +49,14 @@ public class WriterRecordBatch extends AbstractRecordBatch { private long counter = 0; private final RecordBatch incoming; private boolean processed = false; - private String fragmentUniqueId; + private final String fragmentUniqueId; private BatchSchema schema; public WriterRecordBatch(Writer writer, RecordBatch incoming, FragmentContext context, RecordWriter recordWriter) throws OutOfMemoryException { super(writer, context, false); this.incoming = incoming; - FragmentHandle handle = context.getHandle(); + final FragmentHandle handle = context.getHandle(); fragmentUniqueId = String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId()); this.recordWriter = recordWriter; } @@ -78,7 +78,7 @@ public BatchSchema getSchema() { @Override public IterOutcome innerNext() { - if(processed) { + if (processed) { // cleanup(); // if the upstream record batch is already processed and next() is called by // downstream then return NONE to indicate completion @@ -87,7 +87,7 @@ public IterOutcome innerNext() { // process the complete upstream in one next() call IterOutcome upstream; - try{ + try { do { upstream = next(incoming); @@ -129,14 +129,15 @@ public IterOutcome innerNext() { return IterOutcome.OK_NEW_SCHEMA; } - private void addOutputContainerData(){ - VarCharVector fragmentIdVector = (VarCharVector) container.getValueAccessorById( // - VarCharVector.class, // - container.getValueVectorId(SchemaPath.getSimplePath("Fragment")).getFieldIds() // - ).getValueVector(); + private void addOutputContainerData() { + final VarCharVector fragmentIdVector = (VarCharVector) container.getValueAccessorById( + VarCharVector.class, + container.getValueVectorId(SchemaPath.getSimplePath("Fragment")).getFieldIds()) + .getValueVector(); AllocationHelper.allocate(fragmentIdVector, 1, 50); - BigIntVector summaryVector = (BigIntVector) container.getValueAccessorById(BigIntVector.class, - container.getValueVectorId(SchemaPath.getSimplePath("Number of records written")).getFieldIds()).getValueVector(); + final BigIntVector summaryVector = (BigIntVector) container.getValueAccessorById(BigIntVector.class, + container.getValueVectorId(SchemaPath.getSimplePath("Number of records written")).getFieldIds()) + .getValueVector(); AllocationHelper.allocate(summaryVector, 1, 8); fragmentIdVector.getMutator().setSafe(0, fragmentUniqueId.getBytes()); fragmentIdVector.getMutator().setValueCount(1); @@ -154,13 +155,16 @@ protected void setupNewSchema() throws IOException { // Create two vectors for: // 1. Fragment unique id. // 2. Summary: currently contains number of records written. - MaterializedField fragmentIdField = MaterializedField.create(SchemaPath.getSimplePath("Fragment"), Types.required(MinorType.VARCHAR)); - MaterializedField summaryField = MaterializedField.create(SchemaPath.getSimplePath("Number of records written"), Types.required(MinorType.BIGINT)); + final MaterializedField fragmentIdField = + MaterializedField.create(SchemaPath.getSimplePath("Fragment"), Types.required(MinorType.VARCHAR)); + final MaterializedField summaryField = + MaterializedField.create(SchemaPath.getSimplePath("Number of records written"), + Types.required(MinorType.BIGINT)); container.addOrGet(fragmentIdField); container.addOrGet(summaryField); container.buildSchema(BatchSchema.SelectionVectorMode.NONE); - } finally{ + } finally { stats.stopSetup(); } @@ -170,7 +174,7 @@ protected void setupNewSchema() throws IOException { } @Override - public void close() { + public void close() throws Exception { try { if (recordWriter != null) { recordWriter.cleanup(); @@ -181,5 +185,4 @@ public void close() { } super.close(); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java index a033a8ea050..422304e3e78 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java @@ -19,6 +19,7 @@ import java.io.IOException; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.exceptions.ExecutionSetupException; import org.apache.drill.common.exceptions.UserException; import org.apache.drill.common.expression.ErrorCollector; @@ -31,7 +32,6 @@ import org.apache.drill.exec.exception.ClassTransformationException; import org.apache.drill.exec.exception.SchemaChangeException; import org.apache.drill.exec.expr.ClassGenerator; -import org.apache.drill.exec.expr.ClassGenerator.HoldingContainer; import org.apache.drill.exec.expr.CodeGenerator; import org.apache.drill.exec.expr.ExpressionTreeMaterializer; import org.apache.drill.exec.expr.TypeHelper; @@ -56,9 +56,10 @@ import com.sun.codemodel.JVar; public class HashAggBatch extends AbstractRecordBatch { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashAggBatch.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashAggBatch.class); private HashAggregator aggregator; + private final RecordBatch incoming; private LogicalExpression[] aggrExprs; private TypedFieldId[] groupByOutFieldIds; @@ -110,7 +111,7 @@ public void buildSchema() throws SchemaChangeException { if (!createAggregator()) { state = BatchState.DONE; } - for (VectorWrapper w : container) { + for (VectorWrapper w : container) { AllocationHelper.allocatePrecomputedChildCount(w.getValueVector(), 0, 0, 0); } } @@ -142,7 +143,7 @@ public IterOutcome innerNext() { case UPDATE_AGGREGATOR: context.fail(UserException.unsupportedError() .message("Hash aggregate does not support schema changes").build(logger)); - close(); + DrillAutoCloseables.closeNoChecked(this); killIncoming(false); return IterOutcome.STOP; default: @@ -200,7 +201,7 @@ private HashAggregator createAggregatorInternal() throws SchemaChangeException, } final MaterializedField outputField = MaterializedField.create(ne.getRef(), expr.getMajorType()); - ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator()); + final ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator()); // add this group-by vector to the output container groupByOutFieldIds[i] = container.add(vv); @@ -220,7 +221,7 @@ private HashAggregator createAggregatorInternal() throws SchemaChangeException, } final MaterializedField outputField = MaterializedField.create(ne.getRef(), expr.getMajorType()); - ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator()); + final ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator()); aggrOutFieldIds[i] = container.add(vv); aggrExprs[i] = new ValueVectorWriteExpression(aggrOutFieldIds[i], expr, true); @@ -249,9 +250,8 @@ private HashAggregator createAggregatorInternal() throws SchemaChangeException, private void setupUpdateAggrValues(ClassGenerator cg) { cg.setMappingSet(UpdateAggrValuesMapping); - - for (LogicalExpression aggr : aggrExprs) { - HoldingContainer hc = cg.addExpr(aggr); + for (final LogicalExpression aggr : aggrExprs) { + cg.addExpr(aggr); } } @@ -279,7 +279,7 @@ private void setupGetIndex(ClassGenerator cg) { } @Override - public void close() { + public void close() throws Exception { if (aggregator != null) { aggregator.cleanup(); } @@ -290,5 +290,4 @@ public void close() { protected void killIncoming(boolean sendUpstream) { incoming.kill(sendUpstream); } - } 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 8af15082dff..2b2c9ce3fb5 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 @@ -24,8 +24,7 @@ import javax.inject.Named; -import org.apache.drill.common.expression.ErrorCollector; -import org.apache.drill.common.expression.ErrorCollectorImpl; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.expression.ExpressionPosition; import org.apache.drill.common.expression.FieldReference; import org.apache.drill.common.expression.LogicalExpression; @@ -44,7 +43,6 @@ import org.apache.drill.exec.physical.impl.common.HashTableConfig; import org.apache.drill.exec.physical.impl.common.HashTableStats; import org.apache.drill.exec.physical.impl.common.IndexPointer; -import org.apache.drill.exec.record.BatchSchema; import org.apache.drill.exec.record.MaterializedField; import org.apache.drill.exec.record.RecordBatch; import org.apache.drill.exec.record.RecordBatch.IterOutcome; @@ -60,30 +58,21 @@ public abstract class HashAggTemplate implements HashAggregator { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashAggregator.class); - private static final long ALLOCATOR_INITIAL_RESERVATION = 1 * 1024 * 1024; - private static final long ALLOCATOR_MAX_RESERVATION = 20L * 1000 * 1000 * 1000; private static final int VARIABLE_WIDTH_VALUE_SIZE = 50; private static final boolean EXTRA_DEBUG_1 = false; private static final boolean EXTRA_DEBUG_2 = false; - private static final String TOO_BIG_ERROR = - "Couldn't add value to an empty batch. This likely means that a single value is too long for a varlen field."; - private boolean newSchema = false; private int underlyingIndex = 0; private int currentIndex = 0; private IterOutcome outcome; - private int outputCount = 0; private int numGroupedRecords = 0; private int outBatchIndex = 0; private int lastBatchOutputCount = 0; private RecordBatch incoming; - private BatchSchema schema; private HashAggBatch outgoing; private VectorContainer outContainer; - private FragmentContext context; private BufferAllocator allocator; - private HashAggregate hashAggrConfig; private HashTable htable; private ArrayList batchHolders; private IndexPointer htIdxHolder; // holder for the Hashtable's internal index returned by put() @@ -91,8 +80,6 @@ public abstract class HashAggTemplate implements HashAggregator { private IndexPointer outNumRecordsHolder; private int numGroupByOutFields = 0; // Note: this should be <= number of group-by fields - ErrorCollector collector = new ErrorCollectorImpl(); - private MaterializedField[] materializedValueFields; private boolean allFlushed = false; private boolean buildComplete = false; @@ -101,7 +88,6 @@ public abstract class HashAggTemplate implements HashAggregator { private HashTableStats htStats = new HashTableStats(); public enum Metric implements MetricDef { - NUM_BUCKETS, NUM_ENTRIES, NUM_RESIZING, @@ -117,8 +103,7 @@ public int metricId() { public class BatchHolder { - - private VectorContainer aggrValuesContainer; // container for aggr values (workspace variables) + private final VectorContainer aggrValuesContainer; // container for aggr values (workspace variables) private int maxOccupiedIdx = -1; private int batchOutputCount = 0; @@ -126,7 +111,6 @@ public class BatchHolder { private boolean allocatedNextBatch = false; private BatchHolder() { - aggrValuesContainer = new VectorContainer(); boolean success = false; try { @@ -155,7 +139,6 @@ private BatchHolder() { } capacity = Math.min(capacity, vector.getValueCapacity()); - aggrValuesContainer.add(vector); } success = true; @@ -231,16 +214,12 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme throw new IllegalArgumentException("Wrong number of workspace variables."); } - this.context = context; this.stats = stats; this.allocator = allocator; this.incoming = incoming; - this.schema = incoming.getSchema(); this.outgoing = outgoing; this.outContainer = outContainer; - this.hashAggrConfig = hashAggrConfig; - // currently, hash aggregation is only applicable if there are group-by expressions. // For non-grouped (a.k.a Plain) aggregations that don't involve DISTINCT, there is no // need to create hash table. However, for plain aggregations with DISTINCT .. @@ -325,7 +304,6 @@ public AggOutcome doWork() { if (EXTRA_DEBUG_1) { logger.debug("Received new schema. Batch has {} records.", incoming.getRecordCount()); } - newSchema = true; this.cleanup(); // TODO: new schema case needs to be handled appropriately return AggOutcome.UPDATE_AGGREGATOR; @@ -377,13 +355,12 @@ public AggOutcome doWork() { private void allocateOutgoing(int records) { // Skip the keys and only allocate for outputting the workspace values // (keys will be output through splitAndTransfer) - Iterator> outgoingIter = outContainer.iterator(); + final Iterator> outgoingIter = outContainer.iterator(); for (int i = 0; i < numGroupByOutFields; i++) { outgoingIter.next(); } while (outgoingIter.hasNext()) { - ValueVector vv = outgoingIter.next().getValueVector(); - MajorType type = vv.getField().getType(); + final ValueVector vv = outgoingIter.next().getValueVector(); /* * In build schema we use the allocation model that specifies exact record count @@ -425,14 +402,6 @@ public void cleanup() { } } - private final AggOutcome setOkAndReturn() { - this.outcome = IterOutcome.OK; - for (VectorWrapper v : outgoing) { - v.getValueVector().getMutator().setValueCount(outputCount); - } - return AggOutcome.RETURN_OUTCOME; - } - private final void incIndex() { underlyingIndex++; if (underlyingIndex >= incoming.getRecordCount()) { @@ -458,9 +427,10 @@ private void addBatchHolder() { bh.setup(); } + @Override public IterOutcome outputCurrentBatch() { if (outBatchIndex >= batchHolders.size()) { - this.outcome = IterOutcome.NONE; + outcome = IterOutcome.NONE; return outcome; } @@ -468,7 +438,7 @@ public IterOutcome outputCurrentBatch() { int numPendingOutput = batchHolders.get(outBatchIndex).getNumPendingOutput(); if (numPendingOutput == 0) { - this.outcome = IterOutcome.NONE; + outcome = IterOutcome.NONE; return outcome; } @@ -487,9 +457,7 @@ public IterOutcome outputCurrentBatch() { v.getValueVector().getMutator().setValueCount(numOutputRecords); } - outputCount += numOutputRecords; - - this.outcome = IterOutcome.OK; + outcome = IterOutcome.OK; logger.debug("HashAggregate: Output current batch index {} with {} records.", outBatchIndex, numOutputRecords); @@ -504,13 +472,15 @@ public IterOutcome outputCurrentBatch() { this.cleanup(); } - return this.outcome; + return outcome; } + @Override public boolean allFlushed() { return allFlushed; } + @Override public boolean buildComplete() { return buildComplete; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java index 5a26134e689..a66434a9f99 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java @@ -19,6 +19,7 @@ import java.io.IOException; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.drill.common.exceptions.UserException; import org.apache.drill.common.expression.ErrorCollector; @@ -58,7 +59,7 @@ import com.sun.codemodel.JVar; public class StreamingAggBatch extends AbstractRecordBatch { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamingAggBatch.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamingAggBatch.class); private StreamingAggregator aggregator; private final RecordBatch incoming; @@ -113,14 +114,13 @@ public void buildSchema() throws SchemaChangeException { if (!createAggregator()) { state = BatchState.DONE; } - for (VectorWrapper w : container) { + for (VectorWrapper w : container) { w.getValueVector().allocateNew(); } } @Override public IterOutcome innerNext() { - // if a special batch has been sent, we have no data in the incoming so exit early if (specialBatchSent) { return IterOutcome.NONE; @@ -189,7 +189,7 @@ public IterOutcome innerNext() { context.fail(UserException.unsupportedError() .message("Streaming aggregate does not support schema changes") .build(logger)); - close(); + DrillAutoCloseables.closeNoChecked(this); killIncoming(false); return IterOutcome.STOP; default: @@ -206,8 +206,8 @@ public IterOutcome innerNext() { */ private void constructSpecialBatch() { int exprIndex = 0; - for (VectorWrapper vw: container) { - ValueVector vv = vw.getValueVector(); + for (final VectorWrapper vw: container) { + final ValueVector vv = vw.getValueVector(); AllocationHelper.allocateNew(vv, SPECIAL_BATCH_COUNT); vv.getMutator().setValueCount(SPECIAL_BATCH_COUNT); if (vv.getField().getType().getMode() == TypeProtos.DataMode.REQUIRED) { @@ -273,7 +273,7 @@ private StreamingAggregator createAggregatorInternal() throws SchemaChangeExcept } keyExprs[i] = expr; final MaterializedField outputField = MaterializedField.create(ne.getRef(), expr.getMajorType()); - ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator()); + final ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator()); keyOutputIds[i] = container.add(vector); } @@ -285,7 +285,7 @@ private StreamingAggregator createAggregatorInternal() throws SchemaChangeExcept } final MaterializedField outputField = MaterializedField.create(ne.getRef(), expr.getMajorType()); - ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator()); + final ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator()); TypedFieldId id = container.add(vector); valueExprs[i] = new ValueVectorWriteExpression(id, expr, true); } @@ -361,7 +361,7 @@ private void setupIsSameApart(ClassGenerator cg, LogicalExp private void addRecordValues(ClassGenerator cg, LogicalExpression[] valueExprs) { cg.setMappingSet(EVAL); for (LogicalExpression ex : valueExprs) { - HoldingContainer hc = cg.addExpr(ex); + cg.addExpr(ex); } } @@ -370,7 +370,7 @@ private void addRecordValues(ClassGenerator cg, LogicalExpr private void outputRecordKeys(ClassGenerator cg, TypedFieldId[] keyOutputIds, LogicalExpression[] keyExprs) { cg.setMappingSet(RECORD_KEYS); for (int i =0; i < keyExprs.length; i++) { - HoldingContainer hc = cg.addExpr(new ValueVectorWriteExpression(keyOutputIds[i], keyExprs[i], true)); + cg.addExpr(new ValueVectorWriteExpression(keyOutputIds[i], keyExprs[i], true)); } } @@ -390,8 +390,7 @@ private void outputRecordKeysPrev(ClassGenerator cg, TypedF cg.setMappingSet(RECORD_KEYS_PREV); HoldingContainer innerExpression = cg.addExpr(keyExprs[i], false); cg.setMappingSet(RECORD_KEYS_PREV_OUT); - HoldingContainer outerExpression = cg.addExpr(new ValueVectorWriteExpression(keyOutputIds[i], new HoldingContainerExpression(innerExpression), true), false); - + cg.addExpr(new ValueVectorWriteExpression(keyOutputIds[i], new HoldingContainerExpression(innerExpression), true), false); } } @@ -419,14 +418,8 @@ private void getIndex(ClassGenerator g) { } } - @Override - public void close() { - super.close(); - } - @Override protected void killIncoming(boolean sendUpstream) { incoming.kill(sendUpstream); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java index c6a07f8aa37..98db0cdba59 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java @@ -41,8 +41,7 @@ * to all nodes is cheaper than merging and computing all the joins in the same node. */ public class BroadcastSenderRootExec extends BaseRootExec { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BroadcastSenderRootExec.class); - private final FragmentContext context; + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BroadcastSenderRootExec.class); private final BroadcastSender config; private final int[][] receivingMinorFragments; private final AccountingDataTunnel[] tunnels; @@ -64,7 +63,6 @@ public BroadcastSenderRootExec(FragmentContext context, BroadcastSender config) throws OutOfMemoryException { super(context, context.newOperatorContext(config, null, false), config); this.ok = true; - this.context = context; this.incoming = incoming; this.config = config; this.handle = context.getHandle(); @@ -80,7 +78,7 @@ public BroadcastSenderRootExec(FragmentContext context, this.tunnels = new AccountingDataTunnel[destCount]; this.receivingMinorFragments = new int[destCount][]; - for(DrillbitEndpoint ep : dests.keySet()){ + for(final DrillbitEndpoint ep : dests.keySet()){ List minorsList= dests.get(ep); int[] minorsArray = new int[minorsList.size()]; int x = 0; @@ -154,5 +152,4 @@ public void updateStats(FragmentWritableBatch writableBatch) { stats.setLongStat(Metric.N_RECEIVERS, tunnels.length); stats.addLongStat(Metric.BYTES_SENT, writableBatch.getByteCount()); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java index 5eee9dfe5f4..e8daea95d7c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java @@ -28,7 +28,6 @@ import org.apache.drill.exec.expr.ClassGenerator; import org.apache.drill.exec.expr.CodeGenerator; import org.apache.drill.exec.expr.ExpressionTreeMaterializer; -import org.apache.drill.exec.memory.BufferAllocator; import org.apache.drill.exec.memory.OutOfMemoryException; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.config.Filter; @@ -44,11 +43,11 @@ import com.google.common.collect.Lists; public class FilterRecordBatch extends AbstractSingleRecordBatch{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class); private SelectionVector2 sv2; private SelectionVector4 sv4; - private BufferAllocator.PreAllocator svAllocator; +// private BufferAllocator.PreAllocator svAllocator; private Filterer filter; public FilterRecordBatch(Filter pop, RecordBatch incoming, FragmentContext context) throws OutOfMemoryException { @@ -84,9 +83,8 @@ protected IterOutcome doWork() { return IterOutcome.OK; } - @Override - public void close() { + public void close() throws Exception { if (sv2 != null) { sv2.clear(); } @@ -158,8 +156,8 @@ protected Filterer generateSV4Filterer() throws SchemaChangeException { // allocators.add(getAllocator4(v)); // } - for (VectorWrapper vw : incoming) { - for (ValueVector vv : vw.getValueVectors()) { + for (final VectorWrapper vw : incoming) { + for (final ValueVector vv : vw.getValueVectors()) { TransferPair pair = vv.getTransferPair(); container.add(pair.getTo()); transfers.add(pair); @@ -192,12 +190,11 @@ protected Filterer generateSV2Filterer() throws SchemaChangeException { cg.addExpr(new ReturnValueExpression(expr)); - for (VectorWrapper v : incoming) { + for (final VectorWrapper v : incoming) { TransferPair pair = v.getValueVector().makeTransferPair(container.addOrGet(v.getField())); transfers.add(pair); } - try { TransferPair[] tx = transfers.toArray(new TransferPair[transfers.size()]); Filterer filter = context.getImplementationClass(cg); @@ -206,7 +203,5 @@ protected Filterer generateSV2Filterer() throws SchemaChangeException { } catch (ClassTransformationException | IOException e) { throw new SchemaChangeException("Failure while attempting to load generated class", e); } - } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java index 73f34356869..afa7fc13a04 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java @@ -186,8 +186,8 @@ protected void buildSchema() throws SchemaChangeException { hjHelper = new HashJoinHelper(context, oContext.getAllocator()); try { rightSchema = right.getSchema(); - VectorContainer vectors = new VectorContainer(oContext); - for (VectorWrapper w : right) { + final VectorContainer vectors = new VectorContainer(oContext); + for (VectorWrapper w : right) { vectors.addOrGet(w.getField()); } vectors.buildSchema(SelectionVectorMode.NONE); @@ -198,7 +198,7 @@ protected void buildSchema() throws SchemaChangeException { setupHashTable(); hashJoinProbe = setupHashJoinProbe(); // Build the container schema and set the counts - for (VectorWrapper w : container) { + for (VectorWrapper w : container) { w.getValueVector().allocateNew(); } container.buildSchema(BatchSchema.SelectionVectorMode.NONE); @@ -322,12 +322,11 @@ public void setupHashTable() throws IOException, SchemaChangeException, ClassTra } public void executeBuildPhase() throws SchemaChangeException, ClassTransformationException, IOException { - //Setup the underlying hash table // skip first batch if count is zero, as it may be an empty schema batch if (right.getRecordCount() == 0) { - for (VectorWrapper w : right) { + for (VectorWrapper w : right) { w.clear(); } rightUpstream = next(right); @@ -338,7 +337,6 @@ public void executeBuildPhase() throws SchemaChangeException, ClassTransformatio while (moreData) { switch (rightUpstream) { - case OUT_OF_MEMORY: case NONE: case NOT_YET: @@ -464,7 +462,6 @@ public HashJoinProbe setupHashJoinProbe() throws ClassTransformationException, I int outputFieldId = fieldId; fieldId = 0; JExpression probeIndex = JExpr.direct("probeIndex"); - int recordCount = 0; if (leftUpstream == IterOutcome.OK || leftUpstream == IterOutcome.OK_NEW_SCHEMA) { for (VectorWrapper vv : left) { @@ -481,7 +478,7 @@ public HashJoinProbe setupHashJoinProbe() throws ClassTransformationException, I outputType = inputType; } - ValueVector v = container.addOrGet(MaterializedField.create(vv.getField().getPath(), outputType)); + final ValueVector v = container.addOrGet(MaterializedField.create(vv.getField().getPath(), outputType)); if (v instanceof AbstractContainerVector) { vv.getValueVector().makeTransferPair(v); v.clear(); @@ -495,7 +492,6 @@ public HashJoinProbe setupHashJoinProbe() throws ClassTransformationException, I fieldId++; outputFieldId++; } - recordCount = left.getRecordCount(); } HashJoinProbe hj = context.getImplementationClass(cg); @@ -536,7 +532,7 @@ public void killIncoming(boolean sendUpstream) { } @Override - public void close() { + public void close() throws Exception { if (hjHelper != null) { hjHelper.clear(); } @@ -551,5 +547,4 @@ public void close() { } super.close(); } - } 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 ee2ce7fdc1e..0e0500d8abf 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 @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.List; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.expression.ErrorCollector; import org.apache.drill.common.expression.ErrorCollectorImpl; import org.apache.drill.common.expression.LogicalExpression; @@ -68,7 +69,6 @@ * A merge join combining to incoming in-order batches. */ public class MergeJoinBatch extends AbstractRecordBatch { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MergeJoinBatch.class); public static final long ALLOCATOR_INITIAL_RESERVATION = 1*1024*1024; @@ -110,7 +110,7 @@ public class MergeJoinBatch extends AbstractRecordBatch { private final List conditions; private final JoinRelType joinType; private JoinWorker worker; - public MergeJoinBatchBuilder batchBuilder; + MergeJoinBatchBuilder batchBuilder; // used by the template private boolean areNullsEqual = false; // whether nulls compare equal private static final String LEFT_INPUT = "LEFT INPUT"; @@ -124,10 +124,10 @@ protected MergeJoinBatch(MergeJoinPOP popConfig, FragmentContext context, Record } this.left = left; this.right = right; - this.joinType = popConfig.getJoinType(); - this.status = new JoinStatus(left, right, this); - this.batchBuilder = new MergeJoinBatchBuilder(oContext.getAllocator(), status); - this.conditions = popConfig.getConditions(); + joinType = popConfig.getJoinType(); + status = new JoinStatus(left, right, this); + batchBuilder = new MergeJoinBatchBuilder(oContext.getAllocator(), status); + conditions = popConfig.getConditions(); JoinComparator comparator = JoinComparator.NONE; for (JoinCondition condition : conditions) { @@ -137,6 +137,12 @@ protected MergeJoinBatch(MergeJoinPOP popConfig, FragmentContext context, Record areNullsEqual = (comparator == JoinComparator.IS_NOT_DISTINCT_FROM); } + @Override + public void close() throws Exception { + batchBuilder.close(); + super.close(); + } + public JoinRelType getJoinType() { return joinType; } @@ -249,14 +255,14 @@ public IterOutcome innerNext() { } private void setRecordCountInContainer() { - for (VectorWrapper vw : container) { + for (VectorWrapper vw : container) { Preconditions.checkArgument(!vw.isHyper()); vw.getValueVector().getMutator().setValueCount(getRecordCount()); } } public void resetBatchBuilder() { - batchBuilder.close(); + DrillAutoCloseables.closeNoChecked(batchBuilder); batchBuilder = new MergeJoinBatchBuilder(oContext.getAllocator(), status); } @@ -493,7 +499,7 @@ private void allocateBatch(boolean newSchema) { } } - for (VectorWrapper w : container) { + for (VectorWrapper w : container) { AllocationHelper.allocateNew(w.getValueVector(), Character.MAX_VALUE); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatchBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatchBuilder.java index 27980102206..bca37608875 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatchBuilder.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatchBuilder.java @@ -22,8 +22,8 @@ import java.util.List; import org.apache.drill.exec.exception.SchemaChangeException; +import org.apache.drill.exec.memory.AllocationReservation; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.BufferAllocator.PreAllocator; import org.apache.drill.exec.physical.impl.sort.RecordBatchData; import org.apache.drill.exec.record.BatchSchema; import org.apache.drill.exec.record.MaterializedField; @@ -36,23 +36,21 @@ import com.google.common.collect.ArrayListMultimap; public class MergeJoinBatchBuilder implements AutoCloseable { - private final ArrayListMultimap queuedRightBatches = ArrayListMultimap.create(); - private VectorContainer container; - private int runningBytes; - private int runningBatches; - private int recordCount; - private PreAllocator svAllocator; - private boolean svAllocatorUsed = false; - private JoinStatus status; - - public MergeJoinBatchBuilder(BufferAllocator allocator, JoinStatus status) { - this.container = new VectorContainer(); + private final VectorContainer container; + private int runningBytes = 0; + private int runningBatches = 0; + private int recordCount = 0; + private final AllocationReservation allocationReservation; + private final JoinStatus status; + + public MergeJoinBatchBuilder(final BufferAllocator allocator, final JoinStatus status) { + container = new VectorContainer(); this.status = status; - this.svAllocator = allocator.getNewPreAllocator(); + allocationReservation = allocator.newReservation(); } - public boolean add(RecordBatch batch) { + public boolean add(final RecordBatch batch) { if (batch.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.FOUR_BYTE) { throw new UnsupportedOperationException("A merge join cannot currently work against a sv4 batch."); } @@ -61,26 +59,27 @@ public boolean add(RecordBatch batch) { } // resource checks - long batchBytes = getSize(batch); + final long batchBytes = getSize(batch); if (batchBytes + runningBytes > Integer.MAX_VALUE) { return false; // TODO: 2GB is arbitrary } - if (runningBatches++ >= Character.MAX_VALUE) { + if (runningBatches + 1 >= Character.MAX_VALUE) { return false; // allowed in batch. } - if (!svAllocator.preAllocate(batch.getRecordCount()*4)) { - return false; // sv allocation available. + if (!allocationReservation.add(batch.getRecordCount() * 4)) { + return false; // requested allocation unavailable. } // transfer VVs to a new RecordBatchData - RecordBatchData bd = new RecordBatchData(batch); + final RecordBatchData bd = new RecordBatchData(batch); + ++runningBatches; runningBytes += batchBytes; queuedRightBatches.put(batch.getSchema(), bd); recordCount += bd.getRecordCount(); return true; } - private long getSize(RecordBatch batch) { + private long getSize(final RecordBatch batch) { long bytes = 0; for (VectorWrapper v : batch) { bytes += v.getValueVector().getBufferSize(); @@ -93,11 +92,10 @@ public void build() throws SchemaChangeException { if (queuedRightBatches.size() > Character.MAX_VALUE) { throw new SchemaChangeException("Join cannot work on more than %d batches at a time.", (int) Character.MAX_VALUE); } - final DrillBuf drillBuf = svAllocator.getAllocation(); - svAllocatorUsed = true; + final DrillBuf drillBuf = allocationReservation.buffer(); status.sv4 = new SelectionVector4(drillBuf, recordCount, Character.MAX_VALUE); - BatchSchema schema = queuedRightBatches.keySet().iterator().next(); - List data = queuedRightBatches.get(schema); + final BatchSchema schema = queuedRightBatches.keySet().iterator().next(); + final List data = queuedRightBatches.get(schema); // now we're going to generate the sv4 pointers switch (schema.getSelectionVectorMode()) { @@ -130,7 +128,7 @@ public void build() throws SchemaChangeException { } // next, we'll create lists of each of the vector types. - ArrayListMultimap vectors = ArrayListMultimap.create(); + final ArrayListMultimap vectors = ArrayListMultimap.create(); for (RecordBatchData rbd : queuedRightBatches.values()) { for (ValueVector v : rbd.getVectors()) { vectors.put(v.getField(), v); @@ -138,7 +136,7 @@ public void build() throws SchemaChangeException { } for (MaterializedField f : vectors.keySet()) { - List v = vectors.get(f); + final List v = vectors.get(f); container.addHyperList(v); } @@ -146,12 +144,7 @@ public void build() throws SchemaChangeException { } @Override - public void close() { - if (!svAllocatorUsed) { - final DrillBuf drillBuf = svAllocator.getAllocation(); - if (drillBuf != null) { - drillBuf.release(); - } - } + public void close() throws Exception { + allocationReservation.close(); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java index 2d37fa56951..0708776aa7b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java @@ -41,7 +41,6 @@ import org.apache.drill.exec.record.VectorWrapper; import org.apache.drill.exec.server.options.DrillConfigIterator.Iter; import org.apache.drill.exec.vector.AllocationHelper; - import com.google.common.base.Preconditions; import com.sun.codemodel.JExpr; import com.sun.codemodel.JExpression; @@ -183,7 +182,7 @@ public IterOutcome innerNext() { outputRecords = nljWorker.outputRecords(); // Set the record count - for (VectorWrapper vw : container) { + for (VectorWrapper vw : container) { vw.getValueVector().getMutator().setValueCount(outputRecords); } @@ -280,7 +279,7 @@ private NestedLoopJoin setupWorker() throws IOException, ClassTransformationExce * Simple method to allocate space for all the vectors in the container. */ private void allocateVectors() { - for (VectorWrapper vw : container) { + for (VectorWrapper vw : container) { AllocationHelper.allocateNew(vw.getValueVector(), MAX_BATCH_SIZE); } } @@ -309,7 +308,7 @@ protected void buildSchema() throws SchemaChangeException { if (leftUpstream != IterOutcome.NONE) { leftSchema = left.getSchema(); - for (VectorWrapper vw : left) { + for (VectorWrapper vw : left) { container.addOrGet(vw.getField()); } @@ -321,7 +320,7 @@ protected void buildSchema() throws SchemaChangeException { if (rightUpstream != IterOutcome.NONE) { rightSchema = right.getSchema(); - for (VectorWrapper vw : right) { + for (VectorWrapper vw : right) { container.addOrGet(vw.getField()); } addBatchToHyperContainer(right); @@ -355,7 +354,7 @@ private void addBatchToHyperContainer(RecordBatch inputBatch) { } @Override - public void close() { + public void close() throws Exception { rightContainer.clear(); rightCounts.clear(); super.close(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java index d9330ea753a..f6559fdcd70 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java @@ -33,19 +33,19 @@ import com.google.common.collect.Lists; public class LimitRecordBatch extends AbstractSingleRecordBatch { - -// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LimitRecordBatch.class); + // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LimitRecordBatch.class); private SelectionVector2 outgoingSv; private SelectionVector2 incomingSv; private int recordsToSkip; private int recordsLeft; - private boolean noEndLimit; + private final boolean noEndLimit; private boolean skipBatch; private boolean first = true; - List transfers = Lists.newArrayList(); + private final List transfers = Lists.newArrayList(); - public LimitRecordBatch(Limit popConfig, FragmentContext context, RecordBatch incoming) throws OutOfMemoryException { + public LimitRecordBatch(Limit popConfig, FragmentContext context, RecordBatch incoming) + throws OutOfMemoryException { super(popConfig, context, incoming); outgoingSv = new SelectionVector2(oContext.getAllocator()); recordsToSkip = popConfig.getFirst(); @@ -61,14 +61,12 @@ protected boolean setupNewSchema() throws SchemaChangeException { container.zeroVectors(); transfers.clear(); - - for(VectorWrapper v : incoming){ - TransferPair pair = v.getValueVector().makeTransferPair(container.addOrGet(v.getField())); + for(final VectorWrapper v : incoming) { + final TransferPair pair = v.getValueVector().makeTransferPair(container.addOrGet(v.getField())); transfers.add(pair); } - BatchSchema.SelectionVectorMode svMode = incoming.getSchema().getSelectionVectorMode(); - + final BatchSchema.SelectionVectorMode svMode = incoming.getSchema().getSelectionVectorMode(); switch(svMode){ case NONE: break; @@ -126,7 +124,7 @@ protected IterOutcome doWork() { first = false; } skipBatch = false; - int recordCount = incoming.getRecordCount(); + final int recordCount = incoming.getRecordCount(); if (recordCount == 0) { skipBatch = true; return IterOutcome.OK; @@ -134,7 +132,7 @@ protected IterOutcome doWork() { for(TransferPair tp : transfers) { tp.transfer(); } - if(recordCount <= recordsToSkip) { + if (recordCount <= recordsToSkip) { recordsToSkip -= recordCount; skipBatch = true; } else { @@ -149,8 +147,9 @@ protected IterOutcome doWork() { return IterOutcome.OK; } + // These two functions are identical except for the computation of the index; merge private void limitWithNoSV(int recordCount) { - int offset = Math.max(0, Math.min(recordCount - 1, recordsToSkip)); + final int offset = Math.max(0, Math.min(recordCount - 1, recordsToSkip)); recordsToSkip -= offset; int fetch; @@ -162,15 +161,14 @@ private void limitWithNoSV(int recordCount) { } int svIndex = 0; - for(char i = (char) offset; i < fetch; i++) { + for(char i = (char) offset; i < fetch; svIndex++, i++) { outgoingSv.setIndex(svIndex, i); - svIndex++; } outgoingSv.setRecordCount(svIndex); } private void limitWithSV(int recordCount) { - int offset = Math.max(0, Math.min(recordCount - 1, recordsToSkip)); + final int offset = Math.max(0, Math.min(recordCount - 1, recordsToSkip)); recordsToSkip -= offset; int fetch; @@ -182,10 +180,9 @@ private void limitWithSV(int recordCount) { } int svIndex = 0; - for(int i = offset; i < fetch; i++) { - char index = incomingSv.getIndex(i); + for(int i = offset; i < fetch; svIndex++, i++) { + final char index = incomingSv.getIndex(i); outgoingSv.setIndex(svIndex, index); - svIndex++; } outgoingSv.setRecordCount(svIndex); } @@ -196,9 +193,8 @@ public int getRecordCount() { } @Override - public void close(){ + public void close() throws Exception { outgoingSv.clear(); super.close(); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java index 49e81ec9762..5b332719d45 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.PriorityQueue; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.drill.common.expression.ErrorCollector; import org.apache.drill.common.expression.ErrorCollectorImpl; @@ -113,7 +114,7 @@ public class MergingRecordBatch extends AbstractRecordBatch private long[] inputCounts; private long[] outputCounts; - public static enum Metric implements MetricDef{ + public static enum Metric implements MetricDef { BYTES_RECEIVED, NUM_SENDERS, NEXT_WAIT_NANOS; @@ -128,11 +129,10 @@ public MergingRecordBatch(final FragmentContext context, final MergingReceiverPOP config, final RawFragmentBatchProvider[] fragProviders) throws OutOfMemoryException { super(config, context, true, context.newOperatorContext(config, false)); - //super(config, context); this.fragProviders = fragProviders; this.context = context; - this.outgoingContainer = new VectorContainer(oContext); - this.stats.setLongStat(Metric.NUM_SENDERS, config.getNumSenders()); + outgoingContainer = new VectorContainer(oContext); + stats.setLongStat(Metric.NUM_SENDERS, config.getNumSenders()); this.config = config; this.inputCounts = new long[config.getNumSenders()]; this.outputCounts = new long[config.getNumSenders()]; @@ -292,7 +292,6 @@ public IterOutcome innerNext() { // create the outgoing schema and vector container, and allocate the initial batch final SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(BatchSchema.SelectionVectorMode.NONE); for (final VectorWrapper v : batchLoaders[0]) { - // add field to the output schema bldr.addField(v.getField()); @@ -313,13 +312,15 @@ public IterOutcome innerNext() { } // allocate the priority queue with the generated comparator - this.pqueue = new PriorityQueue<>(fragProviders.length, new Comparator() { - public int compare(final Node node1, final Node node2) { - final int leftIndex = (node1.batchId << 16) + node1.valueIndex; - final int rightIndex = (node2.batchId << 16) + node2.valueIndex; - return merger.doEval(leftIndex, rightIndex); - } - }); + pqueue = new PriorityQueue(fragProviders.length, + new Comparator() { + @Override + public int compare(Node node1, Node node2) { + final int leftIndex = (node1.batchId << 16) + node1.valueIndex; + final int rightIndex = (node2.batchId << 16) + node2.valueIndex; + return merger.doEval(leftIndex, rightIndex); + } + }); // populate the priority queue with initial values for (int b = 0; b < senderCount; ++b) { @@ -434,7 +435,7 @@ public int compare(final Node node1, final Node node2) { } // set the value counts in the outgoing vectors - for (final VectorWrapper vw : outgoingContainer) { + for (final VectorWrapper vw : outgoingContainer) { vw.getValueVector().getMutator().setValueCount(outgoingPosition); } @@ -461,7 +462,7 @@ public BatchSchema getSchema() { } @Override - public void buildSchema() throws SchemaChangeException { + public void buildSchema() { // find frag provider that has data to use to build schema, and put in tempBatchHolder for later use tempBatchHolder = new RawFragmentBatch[fragProviders.length]; int i = 0; @@ -471,6 +472,7 @@ public void buildSchema() throws SchemaChangeException { state = BatchState.DONE; return; } + final RawFragmentBatch batch = getNext(i); if (batch == null) { if (!context.shouldContinue()) { @@ -509,7 +511,7 @@ public void kill(final boolean sendUpstream) { if (sendUpstream) { informSenders(); } else { - close(); + DrillAutoCloseables.closeNoChecked(this); for (final RawFragmentBatchProvider provider : fragProviders) { provider.kill(context); } @@ -536,7 +538,6 @@ private void informSenders() { // TODO: Code duplication. UnorderedReceiverBatch has the same implementation. private class OutcomeListener implements RpcOutcomeListener { - @Override public void failed(final RpcException ex) { logger.warn("Failed to inform upstream that receiver is finished"); @@ -607,7 +608,7 @@ private boolean isSameSchemaAmongBatches(final RecordBatchLoader[] batchLoaders) } private void allocateOutgoing() { - for (final VectorWrapper w : outgoingContainer) { + for (final VectorWrapper w : outgoingContainer) { final ValueVector v = w.getValueVector(); if (v instanceof FixedWidthVector) { AllocationHelper.allocate(v, OUTGOING_BATCH_SIZE, 1); @@ -628,9 +629,9 @@ private void allocateOutgoing() { * @throws SchemaChangeException */ private MergingReceiverGeneratorBase createMerger() throws SchemaChangeException { - try { - final CodeGenerator cg = CodeGenerator.get(MergingReceiverGeneratorBase.TEMPLATE_DEFINITION, context.getFunctionRegistry()); + final CodeGenerator cg = + CodeGenerator.get(MergingReceiverGeneratorBase.TEMPLATE_DEFINITION, context.getFunctionRegistry()); final ClassGenerator g = cg.getRoot(); ExpandableHyperContainer batch = null; @@ -658,13 +659,14 @@ private MergingReceiverGeneratorBase createMerger() throws SchemaChangeException } } + // TODO should these be static? public final MappingSet MAIN_MAPPING = new MappingSet( (String) null, null, ClassGenerator.DEFAULT_SCALAR_MAP, ClassGenerator.DEFAULT_SCALAR_MAP); public final MappingSet LEFT_MAPPING = new MappingSet("leftIndex", null, ClassGenerator.DEFAULT_SCALAR_MAP, ClassGenerator.DEFAULT_SCALAR_MAP); public final MappingSet RIGHT_MAPPING = new MappingSet("rightIndex", null, ClassGenerator.DEFAULT_SCALAR_MAP, ClassGenerator.DEFAULT_SCALAR_MAP); GeneratorMapping COPIER_MAPPING = new GeneratorMapping("doSetup", "doCopy", null, null); public final MappingSet COPIER_MAPPING_SET = new MappingSet(COPIER_MAPPING, COPIER_MAPPING); - private void generateComparisons(final ClassGenerator g, final VectorAccessible batch) throws SchemaChangeException { + private void generateComparisons(final ClassGenerator g, final VectorAccessible batch) throws SchemaChangeException { g.setMappingSet(MAIN_MAPPING); for (final Ordering od : popConfig.getOrderings()) { @@ -720,8 +722,9 @@ private boolean copyRecordToOutgoingBatch(final Node node) { * as a wrapper for the priority queue. */ public class Node { - public int batchId; // incoming batch - public int valueIndex; // value within the batch + public final int batchId; // incoming batch + public final int valueIndex; // value within the batch + Node(final int batchId, final int valueIndex) { this.batchId = batchId; this.valueIndex = valueIndex; @@ -729,7 +732,7 @@ public class Node { } @Override - public void close() { + public void close() throws Exception { outgoingContainer.clear(); if (batchLoaders != null) { for (final RecordBatchLoader rbl : batchLoaders) { @@ -738,7 +741,13 @@ public void close() { } } } + + if (fragProviders != null) { + for (final RawFragmentBatchProvider f : fragProviders) { + f.close(); + } + } + super.close(); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java index 1286fe1ab3d..3568c0d763e 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java @@ -23,6 +23,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.expression.ErrorCollector; import org.apache.drill.common.expression.ErrorCollectorImpl; import org.apache.drill.common.expression.FieldReference; @@ -70,7 +71,6 @@ import org.apache.drill.exec.vector.IntVector; import org.apache.drill.exec.vector.ValueVector; import org.apache.calcite.rel.RelFieldCollation.Direction; -import org.apache.calcite.rel.RelFieldCollation.NullDirection; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -86,21 +86,18 @@ * by PartitionSenderRootExec to determine which bucket to assign each record to. */ public class OrderedPartitionRecordBatch extends AbstractRecordBatch { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OrderedPartitionRecordBatch.class); - - private static final long ALLOCATOR_INITIAL_RESERVATION = 1*1024*1024; - private static final long ALLOCATOR_MAX_RESERVATION = 20L*1000*1000*1000; - - public static final CacheConfig SINGLE_CACHE_CONFIG = CacheConfig // - .newBuilder(CachedVectorContainer.class) // - .name("SINGLE-" + CachedVectorContainer.class.getSimpleName()) // - .mode(SerializationMode.DRILL_SERIALIZIABLE) // - .build(); - public static final CacheConfig MULTI_CACHE_CONFIG = CacheConfig // - .newBuilder(CachedVectorContainer.class) // - .name("MULTI-" + CachedVectorContainer.class.getSimpleName()) // - .mode(SerializationMode.DRILL_SERIALIZIABLE) // - .build(); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OrderedPartitionRecordBatch.class); + + public static final CacheConfig SINGLE_CACHE_CONFIG = + CacheConfig.newBuilder(CachedVectorContainer.class) + .name("SINGLE-" + CachedVectorContainer.class.getSimpleName()) + .mode(SerializationMode.DRILL_SERIALIZIABLE) + .build(); + public static final CacheConfig MULTI_CACHE_CONFIG = + CacheConfig.newBuilder(CachedVectorContainer.class) + .name("MULTI-" + CachedVectorContainer.class.getSimpleName()) + .mode(SerializationMode.DRILL_SERIALIZIABLE) + .build(); public final MappingSet mainMapping = new MappingSet( (String) null, null, ClassGenerator.DEFAULT_CONSTANT_MAP, ClassGenerator.DEFAULT_SCALAR_MAP); @@ -117,11 +114,11 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch batchQueue; private int recordsSampled; - private int sendingMajorFragmentWidth; + private final int sendingMajorFragmentWidth; private boolean startedUnsampledBatches = false; private boolean upstreamNone = false; private int recordCount; @@ -133,48 +130,49 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch sampledIncomingBatches; - public OrderedPartitionRecordBatch(OrderedPartitionSender pop, RecordBatch incoming, FragmentContext context) throws OutOfMemoryException { + public OrderedPartitionRecordBatch(OrderedPartitionSender pop, RecordBatch incoming, FragmentContext context) + throws OutOfMemoryException { super(pop, context); this.incoming = incoming; - this.partitions = pop.getDestinations().size(); - this.sendingMajorFragmentWidth = pop.getSendingWidth(); - this.recordsToSample = pop.getRecordsToSample(); - this.samplingFactor = pop.getSamplingFactor(); - this.completionFactor = pop.getCompletionFactor(); + partitions = pop.getDestinations().size(); + sendingMajorFragmentWidth = pop.getSendingWidth(); + recordsToSample = pop.getRecordsToSample(); + samplingFactor = pop.getSamplingFactor(); + completionFactor = pop.getCompletionFactor(); DistributedCache cache = null; - this.mmap = cache.getMultiMap(MULTI_CACHE_CONFIG); - this.tableMap = cache.getMap(SINGLE_CACHE_CONFIG); + mmap = cache.getMultiMap(MULTI_CACHE_CONFIG); // TODO so this class is apparently never used + tableMap = cache.getMap(SINGLE_CACHE_CONFIG); Preconditions.checkNotNull(tableMap); - this.mapKey = String.format("%s_%d", context.getHandle().getQueryId(), context.getHandle().getMajorFragmentId()); - this.minorFragmentSampleCount = cache.getCounter(mapKey); - - SchemaPath outputPath = popConfig.getRef(); - MaterializedField outputField = MaterializedField.create(outputPath, Types.required(TypeProtos.MinorType.INT)); - this.partitionKeyVector = (IntVector) TypeHelper.getNewVector(outputField, oContext.getAllocator()); + mapKey = String.format("%s_%d", context.getHandle().getQueryId(), context.getHandle().getMajorFragmentId()); + minorFragmentSampleCount = cache.getCounter(mapKey); + final SchemaPath outputPath = popConfig.getRef(); + final MaterializedField outputField = + MaterializedField.create(outputPath, Types.required(TypeProtos.MinorType.INT)); + partitionKeyVector = (IntVector) TypeHelper.getNewVector(outputField, oContext.getAllocator()); } - @Override - public void close() { - super.close(); + public void close() throws Exception { + super.close(); // TODO shouldn't this be last? this.partitionVectors.clear(); this.partitionKeyVector.clear(); } - - private boolean saveSamples() throws SchemaChangeException, ClassTransformationException, IOException { + private boolean saveSamples() throws SchemaChangeException, + ClassTransformationException, IOException { recordsSampled = 0; IterOutcome upstream; - // Start collecting batches until recordsToSample records have been collected + // Start collecting batches until recordsToSample records have been + // collected - SortRecordBatchBuilder builder = new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES); + final SortRecordBatchBuilder builder = new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES); + final VectorContainer containerToCache = new VectorContainer(); WritableBatch batch = null; CachedVectorContainer sampleToSave = null; - VectorContainer containerToCache = new VectorContainer(); try { builder.add(incoming); @@ -197,36 +195,43 @@ private boolean saveSamples() throws SchemaChangeException, ClassTransformationE break; } } - VectorContainer sortedSamples = new VectorContainer(); + + final VectorContainer sortedSamples = new VectorContainer(); builder.build(context, sortedSamples); // Sort the records according the orderings given in the configuration - Sorter sorter = SortBatch.createNewSorter(context, popConfig.getOrderings(), sortedSamples); - SelectionVector4 sv4 = builder.getSv4(); + final Sorter sorter = SortBatch.createNewSorter(context, + popConfig.getOrderings(), sortedSamples); + final SelectionVector4 sv4 = builder.getSv4(); sorter.setup(context, sv4, sortedSamples); sorter.sort(sv4, sortedSamples); - // Project every Nth record to a new vector container, where N = recordsSampled/(samplingFactor * partitions). + // Project every Nth record to a new vector container, where N = + // recordsSampled/(samplingFactor * partitions). // Uses the - // the expressions from the Orderings to populate each column. There is one column for each Ordering in + // the expressions from the Orderings to populate each column. There is + // one column for each Ordering in // popConfig.orderings. - List localAllocationVectors = Lists.newArrayList(); - SampleCopier copier = getCopier(sv4, sortedSamples, containerToCache, popConfig.getOrderings(), localAllocationVectors); + final List localAllocationVectors = Lists.newArrayList(); + final SampleCopier copier = getCopier(sv4, sortedSamples, + containerToCache, popConfig.getOrderings(), localAllocationVectors); int allocationSize = 50; while (true) { - for (ValueVector vv : localAllocationVectors) { - AllocationHelper.allocate(vv, samplingFactor * partitions, allocationSize); + for (final ValueVector vv : localAllocationVectors) { + AllocationHelper.allocate(vv, samplingFactor * partitions, + allocationSize); } - if (copier.copyRecords(recordsSampled / (samplingFactor * partitions), 0, samplingFactor * partitions)) { + if (copier.copyRecords(recordsSampled / (samplingFactor * partitions), + 0, samplingFactor * partitions)) { break; } else { containerToCache.zeroVectors(); allocationSize *= 2; } } - for (VectorWrapper vw : containerToCache) { + for (final VectorWrapper vw : containerToCache) { vw.getValueVector().getMutator().setValueCount(copier.getOutputRecords()); } containerToCache.setRecordCount(copier.getOutputRecords()); @@ -241,7 +246,7 @@ private boolean saveSamples() throws SchemaChangeException, ClassTransformationE this.sampledIncomingBatches = builder.getHeldRecordBatches(); } finally { builder.clear(); - builder.close(); + DrillAutoCloseables.closeNoChecked(builder); if (batch != null) { batch.clear(); } @@ -251,8 +256,6 @@ private boolean saveSamples() throws SchemaChangeException, ClassTransformationE } } return true; - - } /** @@ -341,20 +344,19 @@ private boolean getPartitionVectors() { } private void buildTable() throws SchemaChangeException, ClassTransformationException, IOException { - // Get all samples from distributed map - - SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES); + final SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder( + context.getAllocator(), MAX_SORT_BYTES); final VectorContainer allSamplesContainer = new VectorContainer(); final VectorContainer candidatePartitionTable = new VectorContainer(); CachedVectorContainer wrap = null; try { - for (CachedVectorContainer w : mmap.get(mapKey)) { + for (final CachedVectorContainer w : mmap.get(mapKey)) { containerBuilder.add(w.get()); } containerBuilder.build(context, allSamplesContainer); - List orderDefs = Lists.newArrayList(); + final List orderDefs = Lists.newArrayList(); int i = 0; for (Ordering od : popConfig.getOrderings()) { SchemaPath sp = SchemaPath.getSimplePath("f" + i++); @@ -362,25 +364,25 @@ private void buildTable() throws SchemaChangeException, ClassTransformationExcep } // sort the data incoming samples. - SelectionVector4 newSv4 = containerBuilder.getSv4(); - Sorter sorter = SortBatch.createNewSorter(context, orderDefs, allSamplesContainer); + final SelectionVector4 newSv4 = containerBuilder.getSv4(); + final Sorter sorter = SortBatch.createNewSorter(context, orderDefs, allSamplesContainer); sorter.setup(context, newSv4, allSamplesContainer); sorter.sort(newSv4, allSamplesContainer); // Copy every Nth record from the samples into a candidate partition table, where N = totalSampledRecords/partitions // Attempt to push this to the distributed map. Only the first candidate to get pushed will be used. SampleCopier copier = null; - List localAllocationVectors = Lists.newArrayList(); + final List localAllocationVectors = Lists.newArrayList(); copier = getCopier(newSv4, allSamplesContainer, candidatePartitionTable, orderDefs, localAllocationVectors); int allocationSize = 50; while (true) { - for (ValueVector vv : localAllocationVectors) { + for (final ValueVector vv : localAllocationVectors) { AllocationHelper.allocate(vv, samplingFactor * partitions, allocationSize); } - int skipRecords = containerBuilder.getSv4().getTotalCount() / partitions; + final int skipRecords = containerBuilder.getSv4().getTotalCount() / partitions; if (copier.copyRecords(skipRecords, skipRecords, partitions - 1)) { assert copier.getOutputRecords() == partitions - 1 : String.format("output records: %d partitions: %d", copier.getOutputRecords(), partitions); - for (VectorWrapper vw : candidatePartitionTable) { + for (final VectorWrapper vw : candidatePartitionTable) { vw.getValueVector().getMutator().setValueCount(copier.getOutputRecords()); } break; @@ -390,19 +392,18 @@ private void buildTable() throws SchemaChangeException, ClassTransformationExcep } } candidatePartitionTable.setRecordCount(copier.getOutputRecords()); - WritableBatch batch = WritableBatch.getBatchNoHVWrap(candidatePartitionTable.getRecordCount(), candidatePartitionTable, false); + final WritableBatch batch = WritableBatch.getBatchNoHVWrap(candidatePartitionTable.getRecordCount(), candidatePartitionTable, false); wrap = new CachedVectorContainer(batch, context.getDrillbitContext().getAllocator()); tableMap.putIfAbsent(mapKey + "final", wrap, 1, TimeUnit.MINUTES); } finally { candidatePartitionTable.clear(); allSamplesContainer.clear(); containerBuilder.clear(); - containerBuilder.close(); + DrillAutoCloseables.closeNoChecked(containerBuilder); if (wrap != null) { wrap.clear(); } } - } /** @@ -437,7 +438,7 @@ private SampleCopier getCopier(SelectionVector4 sv4, VectorContainer incoming, V "Failure while trying to materialize incoming schema. Errors:\n %s.", collector.toErrorString())); } - ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator()); + final ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator()); localAllocationVectors.add(vector); TypedFieldId fid = outgoing.add(vector); ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true); @@ -501,7 +502,7 @@ public IterOutcome innerNext() { // If this is the first iteration, we need to generate the partition vectors before we can proceed if (this.first && upstream == IterOutcome.OK_NEW_SCHEMA) { if (!getPartitionVectors()) { - close(); + DrillAutoCloseables.closeNoChecked(this); return IterOutcome.STOP; } @@ -537,7 +538,7 @@ public IterOutcome innerNext() { case NONE: case NOT_YET: case STOP: - close(); + DrillAutoCloseables.closeNoChecked(this); recordCount = 0; return upstream; case OK_NEW_SCHEMA: @@ -630,11 +631,10 @@ protected void setupNewSchema(VectorAccessible batch) throws SchemaChangeExcepti container.buildSchema(batch.getSchema().getSelectionVectorMode()); try { - this.projector = context.getImplementationClass(cg); + projector = context.getImplementationClass(cg); projector.setup(context, batch, this, transfers, partitionVectors, partitions, popConfig.getRef()); } catch (ClassTransformationException | IOException e) { throw new SchemaChangeException("Failure while attempting to load generated class", e); } } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java index 31fc1609a24..6e49e78c398 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java @@ -58,8 +58,7 @@ import com.sun.codemodel.JType; public class PartitionSenderRootExec extends BaseRootExec { - - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionSenderRootExec.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionSenderRootExec.class); private RecordBatch incoming; private HashPartitionSender operator; private PartitionerDecorator partitioner; @@ -105,10 +104,10 @@ public PartitionSenderRootExec(FragmentContext context, this.incoming = incoming; this.operator = operator; this.context = context; - this.outGoingBatchCount = operator.getDestinations().size(); - this.popConfig = operator; - this.remainingReceivers = new AtomicIntegerArray(outGoingBatchCount); - this.remaingReceiverCount = new AtomicInteger(outGoingBatchCount); + outGoingBatchCount = operator.getDestinations().size(); + popConfig = operator; + remainingReceivers = new AtomicIntegerArray(outGoingBatchCount); + remaingReceiverCount = new AtomicInteger(outGoingBatchCount); stats.setLongStat(Metric.N_RECEIVERS, outGoingBatchCount); // Algorithm to figure out number of threads to parallelize output // numberOfRows/sliceTarget/numReceivers/threadfactor @@ -137,7 +136,6 @@ public PartitionSenderRootExec(FragmentContext context, @Override public boolean innerNext() { - if (!ok) { return false; } @@ -332,6 +330,7 @@ public void receivingFragmentFinished(FragmentHandle handle) { } } + @Override public void close() throws Exception { logger.debug("Partition sender stopping."); super.close(); @@ -340,7 +339,6 @@ public void close() throws Exception { updateAggregateStats(); partitioner.clear(); } - } public void sendEmptyBatch(boolean isLast) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/producer/ProducerConsumerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/producer/ProducerConsumerBatch.java index b9a16413363..943e443fae6 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/producer/ProducerConsumerBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/producer/ProducerConsumerBatch.java @@ -108,7 +108,7 @@ private boolean load(final RecordBatchData batch) { return false; } else { container.clear(); - for (final VectorWrapper w : newContainer) { + for (final VectorWrapper w : newContainer) { container.add(w.getValueVector()); } container.buildSchema(SelectionVectorMode.NONE); @@ -118,7 +118,6 @@ private boolean load(final RecordBatchData batch) { } private class Producer implements Runnable { - RecordBatchDataWrapper wrapper; @Override @@ -200,13 +199,13 @@ protected void killIncoming(final boolean sendUpstream) { } @Override - public void close() { + public void close() throws Exception { stop = true; try { cleanUpLatch.await(); } catch (final InterruptedException e) { logger.warn("Interrupted while waiting for producer to clean up first. I will try to clean up now...", e); - // TODO InterruptedException + // TODO we should retry to wait for the latch } finally { super.close(); clearQueue(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java index dea6ba80447..c44620d93f2 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java @@ -64,7 +64,7 @@ public class SortBatch extends AbstractRecordBatch { public SortBatch(Sort popConfig, FragmentContext context, RecordBatch incoming) throws OutOfMemoryException { super(popConfig, context); this.incoming = incoming; - this.builder = new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES); + builder = new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES); } @Override @@ -83,8 +83,7 @@ public SelectionVector4 getSelectionVector4() { } @Override - public void close() { - builder.clear(); + public void close() throws Exception { builder.close(); super.close(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java index 00f1992727b..3b88f979f39 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java @@ -21,10 +21,13 @@ import java.util.List; import io.netty.buffer.DrillBuf; + +import org.apache.drill.common.AutoCloseablePointer; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.drill.exec.exception.SchemaChangeException; +import org.apache.drill.exec.memory.AllocationReservation; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.BufferAllocator.PreAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.record.BatchSchema; import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode; @@ -40,24 +43,23 @@ import com.google.common.collect.Lists; public class SortRecordBatchBuilder implements AutoCloseable { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SortRecordBatchBuilder.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SortRecordBatchBuilder.class); private final ArrayListMultimap batches = ArrayListMultimap.create(); private int recordCount; private long runningBytes; - private long runningBatches; + private long runningBatches; // TODO this is never used for anything real; never modified, conditions false private final long maxBytes; - private SelectionVector4 sv4; - final PreAllocator svAllocator; - private boolean svAllocatorUsed = false; + private final AutoCloseablePointer pSV4 = new AutoCloseablePointer<>(); + private final AllocationReservation allocationReservation; - public SortRecordBatchBuilder(BufferAllocator a, long maxBytes) { + public SortRecordBatchBuilder(final BufferAllocator allocator, final long maxBytes) { this.maxBytes = maxBytes; - this.svAllocator = a.getNewPreAllocator(); + allocationReservation = allocator.newReservation(); } - private long getSize(VectorAccessible batch) { + private long getSize(final VectorAccessible batch) { long bytes = 0; for (VectorWrapper v : batch) { bytes += v.getValueVector().getBufferSize(); @@ -72,69 +74,73 @@ private long getSize(VectorAccessible batch) { * @return True if the requested add completed successfully. Returns false in the case that this builder is full and cannot receive additional packages. * @throws SchemaChangeException */ - public boolean add(VectorAccessible batch) { + // TODO it looks like the two add()s could share a majority of their code in a local private function + public boolean add(final VectorAccessible batch) { if (batch.getSchema().getSelectionVectorMode() == SelectionVectorMode.FOUR_BYTE) { throw new UnsupportedOperationException("A sort cannot currently work against a sv4 batch."); } + if (batch.getRecordCount() == 0 && batches.size() > 0) { return true; // skip over empty record batches. } - long batchBytes = getSize(batch); + final long batchBytes = getSize(batch); if (batchBytes == 0 && batches.size() > 0) { return true; } if (batchBytes + runningBytes > maxBytes) { return false; // enough data memory. } - if (runningBatches+1 > Character.MAX_VALUE) { + if (runningBatches + 1 > Character.MAX_VALUE) { return false; // allowed in batch. } - if (!svAllocator.preAllocate(batch.getRecordCount()*4)) { + + if (!allocationReservation.add(batch.getRecordCount() * 4)) { return false; // sv allocation available. } - - RecordBatchData bd = new RecordBatchData(batch); + final RecordBatchData bd = new RecordBatchData(batch); runningBytes += batchBytes; batches.put(batch.getSchema(), bd); recordCount += bd.getRecordCount(); return true; } - public void add(RecordBatchData rbd) { - long batchBytes = getSize(rbd.getContainer()); + public void add(final RecordBatchData rbd) { + final long batchBytes = getSize(rbd.getContainer()); if (batchBytes == 0 && batches.size() > 0) { return; } - if(batchBytes + runningBytes > maxBytes) { + if (batchBytes + runningBytes > maxBytes) { final String errMsg = String.format("Adding this batch causes the total size to exceed max allowed size. " + "Current runningBytes %d, Incoming batchBytes %d. maxBytes %d", runningBytes, batchBytes, maxBytes); logger.error(errMsg); throw new DrillRuntimeException(errMsg); } - if(runningBatches >= Character.MAX_VALUE) { + + if (runningBatches >= Character.MAX_VALUE) { final String errMsg = String.format("Tried to add more than %d number of batches.", Character.MAX_VALUE); logger.error(errMsg); throw new DrillRuntimeException(errMsg); } - if(!svAllocator.preAllocate(rbd.getRecordCount()*4)) { + + if (!allocationReservation.add(rbd.getRecordCount() * 4)) { final String errMsg = String.format("Failed to pre-allocate memory for SV. " + "Existing recordCount*4 = %d, " + "incoming batch recordCount*4 = %d", recordCount * 4, rbd.getRecordCount() * 4); logger.error(errMsg); throw new DrillRuntimeException(errMsg); } - if (rbd.getRecordCount() == 0 && batches.size() > 0) { rbd.getContainer().zeroVectors(); - SelectionVector2 sv2 = rbd.getSv2(); + final SelectionVector2 sv2 = rbd.getSv2(); if (sv2 != null) { sv2.clear(); } return; } + runningBytes += batchBytes; batches.put(rbd.getContainer().getSchema(), rbd); recordCount += rbd.getRecordCount(); @@ -150,26 +156,28 @@ public boolean isEmpty() { return batches.isEmpty(); } - public void build(FragmentContext context, VectorContainer outputContainer) throws SchemaChangeException{ + public void build(final FragmentContext context, final VectorContainer outputContainer) + throws SchemaChangeException{ outputContainer.clear(); if (batches.keySet().size() > 1) { throw new SchemaChangeException("Sort currently only supports a single schema."); } if (batches.size() > Character.MAX_VALUE) { - throw new SchemaChangeException("Sort cannot work on more than %d batches at a time.", (int) Character.MAX_VALUE); + throw new SchemaChangeException("Sort cannot work on more than %d batches at a time.", + (int) Character.MAX_VALUE); } if (batches.keys().size() < 1) { assert false : "Invalid to have an empty set of batches with no schemas."; } - final DrillBuf svBuffer = svAllocator.getAllocation(); + final DrillBuf svBuffer = allocationReservation.buffer(); if (svBuffer == null) { throw new OutOfMemoryError("Failed to allocate direct memory for SV4 vector in SortRecordBatchBuilder."); } - svAllocatorUsed = true; - sv4 = new SelectionVector4(svBuffer, recordCount, Character.MAX_VALUE); - BatchSchema schema = batches.keySet().iterator().next(); - List data = batches.get(schema); + pSV4.assignNoChecked(new SelectionVector4(svBuffer, recordCount, Character.MAX_VALUE)); + final BatchSchema schema = batches.keySet().iterator().next(); + final List data = batches.get(schema); + final SelectionVector4 sv4 = pSV4.get(); // now we're going to generate the sv4 pointers switch (schema.getSelectionVectorMode()) { @@ -202,7 +210,7 @@ public void build(FragmentContext context, VectorContainer outputContainer) thro } // next, we'll create lists of each of the vector types. - ArrayListMultimap vectors = ArrayListMultimap.create(); + final ArrayListMultimap vectors = ArrayListMultimap.create(); for (RecordBatchData rbd : batches.values()) { for (ValueVector v : rbd.getVectors()) { vectors.put(v.getField(), v); @@ -210,7 +218,7 @@ public void build(FragmentContext context, VectorContainer outputContainer) thro } for (MaterializedField f : schema) { - List v = vectors.get(f); + final List v = vectors.get(f); outputContainer.addHyperList(v, false); } @@ -218,34 +226,23 @@ public void build(FragmentContext context, VectorContainer outputContainer) thro } public SelectionVector4 getSv4() { - return sv4; + return pSV4.get(); } public void clear() { + DrillAutoCloseables.closeNoChecked(allocationReservation); for (RecordBatchData d : batches.values()) { d.container.clear(); } - if (sv4 != null) { - sv4.clear(); - } - } - - @Override - public void close() { - // Don't leak unused pre-allocated memory. - if (!svAllocatorUsed) { - final DrillBuf drillBuf = svAllocator.getAllocation(); - if (drillBuf != null) { - drillBuf.release(); - } - } + batches.clear(); + pSV4.assignNoChecked(null); } public List getHeldRecordBatches() { - ArrayList containerList = Lists.newArrayList(); + final ArrayList containerList = Lists.newArrayList(); for (BatchSchema bs : batches.keySet()) { for (RecordBatchData bd : batches.get(bs)) { - VectorContainer c = bd.getContainer(); + final VectorContainer c = bd.getContainer(); c.setRecordCount(bd.getRecordCount()); containerList.add(c); } @@ -255,14 +252,22 @@ public List getHeldRecordBatches() { } /** - * For given recordcount how muchmemory does SortRecordBatchBuilder needs for its own purpose. This is used in - * ExternalSortBatch to make decisions about whether to spill or not. + * For given record count, return how much memory does SortRecordBatchBuilder + * needs for its own use. This is used in ExternalSortBatch to make decisions about + * whether to spill or not. * * @param recordCount - * @return + * @return number of bytes we need */ public static long memoryNeeded(int recordCount) { // We need 4 bytes (SV4) for each record. return recordCount * 4; } + + @Override + public void close() throws Exception { + clear(); + allocationReservation.close(); + pSV4.close(); + } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java index 57e7b55d802..a52f0f73b09 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java @@ -48,7 +48,8 @@ public class RemovingRecordBatch extends AbstractSingleRecordBatch v : container){ - ValueVector.Mutator m = v.getValueVector().getMutator(); + final ValueVector.Mutator m = v.getValueVector().getMutator(); m.setValueCount(copiedRecords); } hasRemainder = true; remainderIndex = copiedRecords; - this.recordCount = remainderIndex; + recordCount = remainderIndex; } else { recordCount = copiedRecords; for(VectorWrapper v : container){ - ValueVector.Mutator m = v.getValueVector().getMutator(); + final ValueVector.Mutator m = v.getValueVector().getMutator(); m.setValueCount(recordCount); } if (incoming.getSchema().getSelectionVectorMode() != SelectionVectorMode.FOUR_BYTE) { @@ -131,8 +132,8 @@ protected IterOutcome doWork() { } private void handleRemainder() { - int recordCount = incoming.getRecordCount(); - int remainingRecordCount = incoming.getRecordCount() - remainderIndex; + final int recordCount = incoming.getRecordCount(); + final int remainingRecordCount = incoming.getRecordCount() - remainderIndex; int copiedRecords; while((copiedRecords = copier.copyRecords(remainderIndex, remainingRecordCount)) == 0) { logger.debug("Copied zero records. Retrying"); @@ -156,14 +157,14 @@ private void handleRemainder() { if (copiedRecords < remainingRecordCount) { for(VectorWrapper v : container){ - ValueVector.Mutator m = v.getValueVector().getMutator(); + final ValueVector.Mutator m = v.getValueVector().getMutator(); m.setValueCount(copiedRecords); } remainderIndex += copiedRecords; this.recordCount = copiedRecords; } else { for(VectorWrapper v : container){ - ValueVector.Mutator m = v.getValueVector().getMutator(); + final ValueVector.Mutator m = v.getValueVector().getMutator(); m.setValueCount(remainingRecordCount); this.recordCount = remainingRecordCount; } @@ -183,18 +184,17 @@ private void handleRemainder() { } @Override - public void close(){ + public void close() throws Exception { super.close(); } - private class StraightCopier implements Copier{ - - private List pairs = Lists.newArrayList(); + private class StraightCopier implements Copier { + private final List pairs = Lists.newArrayList(); @Override - public void setupRemover(FragmentContext context, RecordBatch incoming, RecordBatch outgoing){ - for(VectorWrapper vv : incoming){ - TransferPair tp = vv.getValueVector().makeTransferPair(container.addOrGet(vv.getField())); + public void setupRemover(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) { + for(final VectorWrapper vv : incoming) { + final TransferPair tp = vv.getValueVector().makeTransferPair(container.addOrGet(vv.getField())); pairs.add(tp); } } @@ -207,26 +207,25 @@ public int copyRecords(int index, int recordCount) { } return recordCount; } - } - private Copier getStraightCopier(){ - StraightCopier copier = new StraightCopier(); + private Copier getStraightCopier() { + final StraightCopier copier = new StraightCopier(); copier.setupRemover(context, incoming, this); return copier; } - private Copier getGenerated2Copier() throws SchemaChangeException{ + private Copier getGenerated2Copier() throws SchemaChangeException { Preconditions.checkArgument(incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.TWO_BYTE); - for(VectorWrapper vv : incoming){ - TransferPair tp = vv.getValueVector().makeTransferPair(container.addOrGet(vv.getField())); + for(VectorWrapper vv : incoming) { + vv.getValueVector().makeTransferPair(container.addOrGet(vv.getField())); } try { final CodeGenerator cg = CodeGenerator.get(Copier.TEMPLATE_DEFINITION2, context.getFunctionRegistry()); CopyUtil.generateCopies(cg.getRoot(), incoming, false); - Copier copier = context.getImplementationClass(cg); + final Copier copier = context.getImplementationClass(cg); copier.setupRemover(context, incoming, this); return copier; @@ -240,17 +239,17 @@ private Copier getGenerated4Copier() throws SchemaChangeException { return getGenerated4Copier(incoming, context, oContext.getAllocator(), container, this); } - public static Copier getGenerated4Copier(RecordBatch batch, FragmentContext context, BufferAllocator allocator, VectorContainer container, RecordBatch outgoing) throws SchemaChangeException{ - + public static Copier getGenerated4Copier(RecordBatch batch, FragmentContext context, + BufferAllocator allocator, VectorContainer container, RecordBatch outgoing) throws SchemaChangeException{ for(VectorWrapper vv : batch){ - ValueVector v = vv.getValueVectors()[0]; + final ValueVector v = vv.getValueVectors()[0]; v.makeTransferPair(container.addOrGet(v.getField())); } try { final CodeGenerator cg = CodeGenerator.get(Copier.TEMPLATE_DEFINITION4, context.getFunctionRegistry()); CopyUtil.generateCopies(cg.getRoot(), batch, true); - Copier copier = context.getImplementationClass(cg); + final Copier copier = context.getImplementationClass(cg); copier.setupRemover(context, batch, outgoing); return copier; @@ -263,7 +262,4 @@ public static Copier getGenerated4Copier(RecordBatch batch, FragmentContext cont public WritableBatch getWritableBatch() { return WritableBatch.get(this); } - - - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java index 78e83d63e17..166d025c077 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java @@ -53,12 +53,12 @@ * batch */ public class TraceRecordBatch extends AbstractSingleRecordBatch { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TraceRecordBatch.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TraceRecordBatch.class); private SelectionVector2 sv = null; /* Tag associated with each trace operator */ - final String traceTag; + private final String traceTag; /* Location where the log should be dumped */ private final String logLocation; @@ -68,16 +68,16 @@ public class TraceRecordBatch extends AbstractSingleRecordBatch { public TraceRecordBatch(Trace pop, RecordBatch incoming, FragmentContext context) throws ExecutionSetupException { super(pop, context, incoming); - this.traceTag = pop.traceTag; + traceTag = pop.traceTag; logLocation = context.getConfig().getString(ExecConstants.TRACE_DUMP_DIRECTORY); - String fileName = getFileName(); + final String fileName = getFileName(); /* Create the log file we will dump to and initialize the file descriptors */ try { - Configuration conf = new Configuration(); + final Configuration conf = new Configuration(); conf.set(FileSystem.FS_DEFAULT_NAME_KEY, context.getConfig().getString(ExecConstants.TRACE_DUMP_FILESYSTEM)); - FileSystem fs = FileSystem.get(conf); + final FileSystem fs = FileSystem.get(conf); /* create the file */ fos = fs.create(new Path(fileName)); @@ -102,14 +102,15 @@ public int getRecordCount() { @Override protected IterOutcome doWork() { - boolean incomingHasSv2 = incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.TWO_BYTE; + final boolean incomingHasSv2 = incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.TWO_BYTE; if (incomingHasSv2) { sv = incoming.getSelectionVector2(); } else { sv = null; } - WritableBatch batch = WritableBatch.getBatchNoHVWrap(incoming.getRecordCount(), incoming, incomingHasSv2); - VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, sv, oContext.getAllocator()); + final WritableBatch batch = WritableBatch.getBatchNoHVWrap( + incoming.getRecordCount(), incoming, incomingHasSv2); + final VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, sv, oContext.getAllocator()); try { wrap.writeToStreamAndRetain(fos); @@ -137,7 +138,7 @@ protected boolean setupNewSchema() throws SchemaChangeException { /* Add all the value vectors in the container */ for (VectorWrapper vv : incoming) { - TransferPair tp = vv.getValueVector().getTransferPair(); + final TransferPair tp = vv.getValueVector().getTransferPair(); container.add(tp.getTo()); } container.buildSchema(incoming.getSchema().getSelectionVectorMode()); @@ -154,7 +155,7 @@ private String getFileName() { } @Override - public void close() { + public void close() throws Exception { /* Release the selection vector */ if (sv != null) { sv.clear(); @@ -166,7 +167,7 @@ public void close() { } catch (IOException e) { logger.error("Unable to close file descriptors for file: " + getFileName()); } + super.close(); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java index 445568b8620..52b4b920102 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java @@ -63,18 +63,19 @@ public class UnionAllRecordBatch extends AbstractRecordBatch { private List outputFields; private UnionAller unionall; - private UnionAllInput unionAllInput; + private final UnionAllInput unionAllInput; private RecordBatch current; private final List transfers = Lists.newArrayList(); private List allocationVectors; - protected SchemaChangeCallBack callBack = new SchemaChangeCallBack(); + protected final SchemaChangeCallBack callBack = new SchemaChangeCallBack(); private int recordCount = 0; private boolean schemaAvailable = false; - public UnionAllRecordBatch(UnionAll config, List children, FragmentContext context) throws OutOfMemoryException { + public UnionAllRecordBatch(UnionAll config, List children, FragmentContext context) + throws OutOfMemoryException { super(config, context, false); - assert (children.size() == 2) : "The number of the operands of Union must be 2"; + assert children.size() == 2 : "The number of the operands of Union must be 2"; unionAllInput = new UnionAllInput(this, children.get(0), children.get(1)); } @@ -83,6 +84,14 @@ public int getRecordCount() { return recordCount; } + @Override + public void kill(boolean sendUpstream) { + if(current != null) { + current.kill(sendUpstream); + current = null; + } + } + @Override protected void killIncoming(boolean sendUpstream) { unionAllInput.getLeftRecordBatch().kill(sendUpstream); @@ -136,14 +145,14 @@ public WritableBatch getWritableBatch() { } private void setValueCount(int count) { - for (ValueVector v : allocationVectors) { - ValueVector.Mutator m = v.getMutator(); + for (final ValueVector v : allocationVectors) { + final ValueVector.Mutator m = v.getMutator(); m.setValueCount(count); } } private boolean doAlloc() { - for (ValueVector v : allocationVectors) { + for (final ValueVector v : allocationVectors) { try { AllocationHelper.allocateNew(v, current.getRecordCount()); } catch (OutOfMemoryRuntimeException ex) { @@ -155,7 +164,7 @@ private boolean doAlloc() { private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException { if (allocationVectors != null) { - for (ValueVector v : allocationVectors) { + for (final ValueVector v : allocationVectors) { v.clear(); } } @@ -166,11 +175,11 @@ private IterOutcome doWork() throws ClassTransformationException, IOException, S final ClassGenerator cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry()); int index = 0; for(VectorWrapper vw : current) { - ValueVector vvIn = vw.getValueVector(); + final ValueVector vvIn = vw.getValueVector(); // get the original input column names - SchemaPath inputPath = vvIn.getField().getPath(); + final SchemaPath inputPath = vvIn.getField().getPath(); // get the renamed column names - SchemaPath outputPath = outputFields.get(index).getPath(); + final SchemaPath outputPath = outputFields.get(index).getPath(); final ErrorCollector collector = new ErrorCollectorImpl(); // According to input data names, Minortypes, Datamodes, choose to @@ -185,9 +194,9 @@ private IterOutcome doWork() throws ClassTransformationException, IOException, S throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema. Errors:\n %s.", collector.toErrorString())); } - ValueVectorReadExpression vectorRead = (ValueVectorReadExpression) expr; - ValueVector vvOut = container.addOrGet(MaterializedField.create(outputPath, vectorRead.getMajorType())); - TransferPair tp = vvIn.makeTransferPair(vvOut); + final ValueVectorReadExpression vectorRead = (ValueVectorReadExpression) expr; + final ValueVector vvOut = container.addOrGet(MaterializedField.create(outputPath, vectorRead.getMajorType())); + final TransferPair tp = vvIn.makeTransferPair(vvOut); transfers.add(tp); // Copy data in order to rename the column } else { @@ -196,11 +205,11 @@ private IterOutcome doWork() throws ClassTransformationException, IOException, S throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema. Errors:\n %s.", collector.toErrorString())); } - MaterializedField outputField = MaterializedField.create(outputPath, expr.getMajorType()); - ValueVector vv = container.addOrGet(outputField, callBack); + final MaterializedField outputField = MaterializedField.create(outputPath, expr.getMajorType()); + final ValueVector vv = container.addOrGet(outputField, callBack); allocationVectors.add(vv); - TypedFieldId fid = container.getValueVectorId(outputField.getPath()); - ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true); + final TypedFieldId fid = container.getValueVectorId(outputField.getPath()); + final ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true); cg.addExpr(write); } // Cast is necessary @@ -230,11 +239,11 @@ private IterOutcome doWork() throws ClassTransformationException, IOException, S } final MaterializedField outputField = MaterializedField.create(outputPath, expr.getMajorType()); - ValueVector vector = container.addOrGet(outputField, callBack); + final ValueVector vector = container.addOrGet(outputField, callBack); allocationVectors.add(vector); - TypedFieldId fid = container.getValueVectorId(outputField.getPath()); + final TypedFieldId fid = container.getValueVectorId(outputField.getPath()); - boolean useSetSafe = !(vector instanceof FixedWidthVector); + final boolean useSetSafe = !(vector instanceof FixedWidthVector); ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe); cg.addExpr(write); } @@ -270,12 +279,14 @@ private void setCurrentRecordBatch(RecordBatch target) { // This method is used by inner class to clear the current record batch private void clearCurrentRecordBatch() { - for(VectorWrapper v: current) { - v.clear(); + if (current != null) { + for(final VectorWrapper v: current) { + v.clear(); + } } } - public static class UnionAllInput { + public static class UnionAllInput implements AutoCloseable { private UnionAllRecordBatch unionAllRecordBatch; private List outputFields; private OneSideInput leftSide; @@ -295,6 +306,10 @@ public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, rightSide = new OneSideInput(right); } + @Override + public void close() throws Exception { + } + public IterOutcome nextBatch() throws SchemaChangeException { if(upstream == RecordBatch.IterOutcome.NOT_YET) { IterOutcome iterLeft = leftSide.nextBatch(); @@ -506,7 +521,7 @@ public RecordBatch getRightRecordBatch() { private class OneSideInput { private IterOutcome upstream = IterOutcome.NOT_YET; - private RecordBatch recordBatch; + private final RecordBatch recordBatch; public OneSideInput(RecordBatch recordBatch) { this.recordBatch = recordBatch; @@ -535,4 +550,4 @@ public IterOutcome nextBatch() { } } } -} \ No newline at end of file +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java index caabfcea3a0..93f2aa7d8c2 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java @@ -32,6 +32,7 @@ import org.apache.drill.exec.physical.MinorFragmentEndpoint; import org.apache.drill.exec.physical.config.UnorderedReceiver; import org.apache.drill.exec.proto.BitControl.FinishedReceiver; +import org.apache.drill.exec.proto.BitData.FragmentRecordBatch; import org.apache.drill.exec.proto.ExecProtos.FragmentHandle; import org.apache.drill.exec.proto.GeneralRPCProtos.Ack; import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef; @@ -74,7 +75,8 @@ public int metricId() { } } - public UnorderedReceiverBatch(final FragmentContext context, final RawFragmentBatchProvider fragProvider, final UnorderedReceiver config) throws OutOfMemoryException { + public UnorderedReceiverBatch(final FragmentContext context, + final RawFragmentBatchProvider fragProvider, final UnorderedReceiver config) throws OutOfMemoryException { this.fragProvider = fragProvider; this.context = context; // In normal case, batchLoader does not require an allocator. However, in case of splitAndTransfer of a value vector, @@ -82,8 +84,9 @@ public UnorderedReceiverBatch(final FragmentContext context, final RawFragmentBa oContext = context.newOperatorContext(config, false); this.batchLoader = new RecordBatchLoader(oContext.getAllocator()); - this.stats = oContext.getStats(); - this.stats.setLongStat(Metric.NUM_SENDERS, config.getNumSenders()); + stats = oContext.getStats(); + stats.setLongStat(Metric.NUM_SENDERS, config.getNumSenders()); + this.config = config; } @@ -151,7 +154,7 @@ private RawFragmentBatch getNextBatch() throws IOException { @Override public IterOutcome next() { stats.startProcessing(); - try{ + try { RawFragmentBatch batch; try { stats.startWait(); @@ -179,7 +182,6 @@ public IterOutcome next() { return IterOutcome.OUT_OF_MEMORY; } - // logger.debug("Next received batch {}", batch); final RecordBatchDef rbd = batch.getHeader().getDef(); @@ -240,7 +242,6 @@ private void informSenders() { // TODO: Code duplication. MergingRecordBatch has the same implementation. private class OutcomeListener implements RpcOutcomeListener { - @Override public void failed(final RpcException ex) { logger.warn("Failed to inform upstream that receiver is finished"); @@ -260,5 +261,4 @@ public void interrupted(final InterruptedException e) { } } } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java index efd155e1f64..e2ac768a04d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java @@ -33,9 +33,9 @@ import org.apache.drill.exec.vector.VectorValidator; public class IteratorValidatorBatchIterator implements CloseableRecordBatch { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IteratorValidatorBatchIterator.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IteratorValidatorBatchIterator.class); - static final boolean VALIDATE_VECTORS = false; + private static final boolean VALIDATE_VECTORS = false; private IterOutcome state = IterOutcome.NOT_YET; private final RecordBatch incoming; @@ -52,10 +52,10 @@ private void validateReadState() { return; default: throw new IllegalStateException( - String - .format( - "You tried to do a batch data read operation when you were in a state of %s. You can only do this type of operation when you are in a state of OK or OK_NEW_SCHEMA.", - state.name())); + String.format( + "You tried to do a batch data read operation when you were in a state of %s. " + + "You can only do this type of operation when you are in a state of OK or OK_NEW_SCHEMA.", + state.name())); } } @@ -121,7 +121,7 @@ public IterOutcome next() { } if (state == IterOutcome.OK || state == IterOutcome.OK_NEW_SCHEMA) { - BatchSchema schema = incoming.getSchema(); + final BatchSchema schema = incoming.getSchema(); if (schema == null) { return state; } @@ -153,7 +153,7 @@ public void close() { @Override public VectorContainer getOutgoingContainer() { - throw new UnsupportedOperationException(String.format(" You should not call getOutgoingContainer() for class %s", this.getClass().getCanonicalName())); + throw new UnsupportedOperationException(String.format( + " You should not call getOutgoingContainer() for class %s", this.getClass().getCanonicalName())); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/window/WindowFrameRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/window/WindowFrameRecordBatch.java index 9c8cfc01a11..79d143d40ec 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/window/WindowFrameRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/window/WindowFrameRecordBatch.java @@ -347,10 +347,10 @@ private void setupIsFunction(final ClassGenerator cg, final List cg, final List cg, List valueExprs) { cg.setMappingSet(eval); - for (LogicalExpression ex : valueExprs) { + for (final LogicalExpression ex : valueExprs) { cg.addExpr(ex); } } @@ -399,7 +399,7 @@ private void cleanup() { } @Override - public void close() { + public void close() throws Exception { cleanup(); super.close(); } 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 4bb15728fdc..89351980db5 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 @@ -25,6 +25,9 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import org.apache.drill.common.AutoCloseablePointer; +import org.apache.drill.common.DeferredException; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.exceptions.UserException; import org.apache.drill.common.expression.ErrorCollector; @@ -73,6 +76,7 @@ import com.google.common.base.Joiner; import com.google.common.base.Stopwatch; +import com.google.common.base.Supplier; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.sun.codemodel.JConditional; @@ -98,12 +102,12 @@ public class ExternalSortBatch extends AbstractRecordBatch { private BatchSchema schema; private SingleBatchSorter sorter; - private SortRecordBatchBuilder builder; private MSorter mSorter; private PriorityQueueCopier copier; private LinkedList batchGroups = Lists.newLinkedList(); private LinkedList spilledBatchGroups = Lists.newLinkedList(); - private SelectionVector4 sv4; + private final AutoCloseablePointer pBuilder = new AutoCloseablePointer<>(); + private final AutoCloseablePointer pSV4 = new AutoCloseablePointer<>(); private FileSystem fs; private int spillCount = 0; private int batchesSinceLastSpill = 0; @@ -121,8 +125,8 @@ public class ExternalSortBatch extends AbstractRecordBatch { public ExternalSortBatch(ExternalSort popConfig, FragmentContext context, RecordBatch incoming) throws OutOfMemoryException { super(popConfig, context, true); this.incoming = incoming; - DrillConfig config = context.getConfig(); - Configuration conf = new Configuration(); + final DrillConfig config = context.getConfig(); + final Configuration conf = new Configuration(); conf.set("fs.default.name", config.getString(ExecConstants.EXTERNAL_SORT_SPILL_FILESYSTEM)); try { this.fs = FileSystem.get(conf); @@ -134,7 +138,7 @@ public ExternalSortBatch(ExternalSort popConfig, FragmentContext context, Record SPILL_DIRECTORIES = config.getStringList(ExecConstants.EXTERNAL_SORT_SPILL_DIRS); dirs = Iterators.cycle(Lists.newArrayList(SPILL_DIRECTORIES)); copierAllocator = oContext.getAllocator().getChildAllocator( - context, PriorityQueueCopier.initialAllocation, PriorityQueueCopier.maxAllocation, true); + context, PriorityQueueCopier.INITIAL_ALLOCATION, PriorityQueueCopier.MAX_ALLOCATION, true); FragmentHandle handle = context.getHandle(); fileName = String.format("%s/major_fragment_%s/minor_fragment_%s/operator_%s", QueryIdHelper.getQueryId(handle.getQueryId()), handle.getMajorFragmentId(), handle.getMinorFragmentId(), popConfig.getOperatorId()); @@ -142,6 +146,7 @@ public ExternalSortBatch(ExternalSort popConfig, FragmentContext context, Record @Override public int getRecordCount() { + final SelectionVector4 sv4 = pSV4.get(); if (sv4 != null) { return sv4.getCount(); } @@ -150,49 +155,61 @@ public int getRecordCount() { @Override public SelectionVector4 getSelectionVector4() { - return sv4; + return pSV4.get(); } @Override - public void close() { - try { - if (batchGroups != null) { - for (BatchGroup group: batchGroups) { - try { + public void close() throws Exception { + final DeferredException deferredException = new DeferredException(new Supplier() { + @Override + public Exception get() { + return new RuntimeException("Error closing resources"); + } + }); + + if (batchGroups != null) { + for(final BatchGroup group : batchGroups) { + deferredException.suppressingClose(new AutoCloseable() { + @Override + public void close() throws Exception { group.cleanup(); - } catch (IOException e) { - throw new RuntimeException(e); } - } - } - } finally { - if (builder != null) { - builder.clear(); - builder.close(); - } - if (sv4 != null) { - sv4.clear(); + }); } - if (copier != null) { - copier.cleanup(); - } - copierAllocator.close(); - super.close(); + } - if(mSorter != null) { - mSorter.clear(); + deferredException.suppressingClose(pBuilder); + deferredException.suppressingClose(pSV4); + deferredException.suppressingClose(copier); + deferredException.suppressingClose(copierAllocator); + deferredException.suppressingClose(new AutoCloseable() { + @Override + public void close() throws Exception { + ExternalSortBatch.super.close(); } + }); + + if (mSorter != null) { + deferredException.suppressingClose(new AutoCloseable() { + @Override + public void close() throws Exception { + mSorter.clear(); + } + }); } + + // This must be last. + deferredException.close(); } @Override public void buildSchema() throws SchemaChangeException { - IterOutcome outcome = next(incoming); + final IterOutcome outcome = next(incoming); switch (outcome) { case OK: case OK_NEW_SCHEMA: - for (VectorWrapper w : incoming) { - ValueVector v = container.addOrGet(w.getField()); + for (VectorWrapper w : incoming) { + final ValueVector v = container.addOrGet(w.getField()); if (v instanceof AbstractContainerVector) { w.getValueVector().makeTransferPair(v); // Can we remove this hack? v.clear(); @@ -220,9 +237,9 @@ public IterOutcome innerNext() { if (spillCount == 0) { return (getSelectionVector4().next()) ? IterOutcome.OK : IterOutcome.NONE; } else { - Stopwatch w = new Stopwatch(); + final Stopwatch w = new Stopwatch(); w.start(); - int count = copier.next(targetRecordCount); + final int count = copier.next(targetRecordCount); if (count > 0) { long t = w.elapsed(TimeUnit.MICROSECONDS); logger.debug("Took {} us to merge {} records", t, count); @@ -240,7 +257,7 @@ public IterOutcome innerNext() { try{ container.clear(); outer: while (true) { - Stopwatch watch = new Stopwatch(); + final Stopwatch watch = new Stopwatch(); watch.start(); IterOutcome upstream; if (first) { @@ -277,17 +294,18 @@ public IterOutcome innerNext() { first = false; } if (incoming.getRecordCount() == 0) { - for (VectorWrapper w : incoming) { + for (VectorWrapper w : incoming) { w.clear(); } break; } totalSizeInMemory += getBufferSize(incoming); - SelectionVector2 sv2; + final SelectionVector2 sv2; if (incoming.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.TWO_BYTE) { sv2 = incoming.getSelectionVector2(); - if (sv2.getBuffer(false).isRootBuffer()) { - oContext.getAllocator().takeOwnership(sv2.getBuffer(false)); + final DrillBuf sv2Buf = sv2.getBuffer(false); + if (sv2Buf.isRootBuffer()) { + oContext.getAllocator().takeOwnership(sv2Buf); } } else { try { @@ -298,14 +316,14 @@ public IterOutcome innerNext() { throw new OutOfMemoryRuntimeException(e); } } - int count = sv2.getCount(); + final int count = sv2.getCount(); totalCount += count; sorter.setup(context, sv2, incoming); - Stopwatch w = new Stopwatch(); + final Stopwatch w = new Stopwatch(); w.start(); sorter.sort(sv2); // logger.debug("Took {} us to sort {} records", w.elapsed(TimeUnit.MICROSECONDS), count); - RecordBatchData rbd = new RecordBatchData(incoming); + final RecordBatchData rbd = new RecordBatchData(incoming); boolean success = false; try { if (incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.NONE) { @@ -364,24 +382,21 @@ public IterOutcome innerNext() { if (totalCount == 0) { return IterOutcome.NONE; } + if (spillCount == 0) { - Stopwatch watch = new Stopwatch(); + final Stopwatch watch = new Stopwatch(); watch.start(); - if (builder != null) { - builder.clear(); - builder.close(); - } - builder = new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES); - + pBuilder.assignNoChecked(new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES)); + final SortRecordBatchBuilder builder = pBuilder.get(); for (BatchGroup group : batchGroups) { - RecordBatchData rbd = new RecordBatchData(group.getContainer()); + final RecordBatchData rbd = new RecordBatchData(group.getContainer()); rbd.setSv2(group.getSv2()); builder.add(rbd); } builder.build(context, container); - sv4 = builder.getSv4(); + pSV4.assignNoChecked(builder.getSv4()); mSorter = createNewMSorter(); mSorter.setup(context, oContext.getAllocator(), getSelectionVector4(), this.container); @@ -396,28 +411,30 @@ public IterOutcome innerNext() { // For testing memory-leak purpose, inject exception after mSorter finishes sorting injector.injectUnchecked(context.getExecutionControls(), INTERRUPTION_AFTER_SORT); - sv4 = mSorter.getSV4(); + pSV4.assignNoChecked(mSorter.getSV4()); - long t = watch.elapsed(TimeUnit.MICROSECONDS); -// logger.debug("Took {} us to sort {} records", t, sv4.getTotalCount()); + final long t = watch.elapsed(TimeUnit.MICROSECONDS); + logger.debug("Took {} us to sort {} records", t, pSV4.get().getTotalCount()); container.buildSchema(SelectionVectorMode.FOUR_BYTE); } else { spilledBatchGroups.add(mergeAndSpill(batchGroups)); batchGroups.addAll(spilledBatchGroups); logger.warn("Starting to merge. {} batch groups. Current allocated memory: {}", batchGroups.size(), oContext.getAllocator().getAllocatedMemory()); - VectorContainer hyperBatch = constructHyperBatch(batchGroups); + + final VectorContainer hyperBatch = constructHyperBatch(batchGroups); createCopier(hyperBatch, batchGroups, container, false); int estimatedRecordSize = 0; - for (VectorWrapper w : batchGroups.get(0)) { + for (VectorWrapper w : batchGroups.get(0)) { try { estimatedRecordSize += TypeHelper.getSize(w.getField().getType()); } catch (UnsupportedOperationException e) { estimatedRecordSize += 50; } } + targetRecordCount = Math.min(MAX_BATCH_SIZE, Math.max(1, 250 * 1000 / estimatedRecordSize)); - int count = copier.next(targetRecordCount); + final int count = copier.next(targetRecordCount); container.buildSchema(SelectionVectorMode.NONE); container.setRecordCount(count); } @@ -449,42 +466,42 @@ private boolean hasMemoryForInMemorySort(int currentRecordCount) { public BatchGroup mergeAndSpill(LinkedList batchGroups) throws SchemaChangeException { logger.debug("Copier allocator current allocation {}", copierAllocator.getAllocatedMemory()); - VectorContainer outputContainer = new VectorContainer(); - List batchGroupList = Lists.newArrayList(); - int batchCount = batchGroups.size(); + final VectorContainer outputContainer = new VectorContainer(); + final List batchGroupList = Lists.newArrayList(); + final int batchCount = batchGroups.size(); for (int i = 0; i < batchCount / 2; i++) { if (batchGroups.size() == 0) { break; } - BatchGroup batch = batchGroups.pollLast(); + final BatchGroup batch = batchGroups.pollLast(); batchGroupList.add(batch); - long bufferSize = getBufferSize(batch); + final long bufferSize = getBufferSize(batch); totalSizeInMemory -= bufferSize; } if (batchGroupList.size() == 0) { return null; } int estimatedRecordSize = 0; - for (VectorWrapper w : batchGroupList.get(0)) { + for (final VectorWrapper w : batchGroupList.get(0)) { try { estimatedRecordSize += TypeHelper.getSize(w.getField().getType()); } catch (UnsupportedOperationException e) { estimatedRecordSize += 50; } } - int targetRecordCount = Math.max(1, 250 * 1000 / estimatedRecordSize); - VectorContainer hyperBatch = constructHyperBatch(batchGroupList); + final int targetRecordCount = Math.max(1, 250 * 1000 / estimatedRecordSize); + final VectorContainer hyperBatch = constructHyperBatch(batchGroupList); createCopier(hyperBatch, batchGroupList, outputContainer, true); int count = copier.next(targetRecordCount); assert count > 0; - VectorContainer c1 = VectorContainer.getTransferClone(outputContainer); + final VectorContainer c1 = VectorContainer.getTransferClone(outputContainer); c1.buildSchema(BatchSchema.SelectionVectorMode.NONE); c1.setRecordCount(count); - String outputFile = Joiner.on("/").join(dirs.next(), fileName, spillCount++); - BatchGroup newGroup = new BatchGroup(c1, fs, outputFile, oContext.getAllocator()); + final String outputFile = Joiner.on("/").join(dirs.next(), fileName, spillCount++); + final BatchGroup newGroup = new BatchGroup(c1, fs, outputFile, oContext.getAllocator()); logger.info("Merging and spilling to {}", outputFile); try { @@ -507,22 +524,23 @@ public BatchGroup mergeAndSpill(LinkedList batchGroups) throws Schem return newGroup; } - private void takeOwnership(VectorAccessible batch) { - for (VectorWrapper w : batch) { - DrillBuf[] bufs = w.getValueVector().getBuffers(false); - for (DrillBuf buf : bufs) { + private void takeOwnership(final VectorAccessible batch) { + final BufferAllocator allocator = oContext.getAllocator(); + for (final VectorWrapper w : batch) { + final DrillBuf[] bufs = w.getValueVector().getBuffers(false); + for (final DrillBuf buf : bufs) { if (buf.isRootBuffer()) { - oContext.getAllocator().takeOwnership(buf); + allocator.takeOwnership(buf); } } } } - private long getBufferSize(VectorAccessible batch) { + private long getBufferSize(final VectorAccessible batch) { long size = 0; - for (VectorWrapper w : batch) { - DrillBuf[] bufs = w.getValueVector().getBuffers(false); - for (DrillBuf buf : bufs) { + for (final VectorWrapper w : batch) { + final DrillBuf[] bufs = w.getValueVector().getBuffers(false); + for (final DrillBuf buf : bufs) { if (buf.isRootBuffer()) { size += buf.capacity(); } @@ -532,12 +550,12 @@ private long getBufferSize(VectorAccessible batch) { } private SelectionVector2 newSV2() throws OutOfMemoryException, InterruptedException { - SelectionVector2 sv2 = new SelectionVector2(oContext.getAllocator()); + final SelectionVector2 sv2 = new SelectionVector2(oContext.getAllocator()); if (!sv2.allocateNewSafe(incoming.getRecordCount())) { try { spilledBatchGroups.addFirst(mergeAndSpill(batchGroups)); } catch (SchemaChangeException e) { - throw new RuntimeException(); + throw new RuntimeException(e); // TODO sv2 will be left unclosed here } batchesSinceLastSpill = 0; int waitTime = 1; @@ -554,7 +572,7 @@ private SelectionVector2 newSV2() throws OutOfMemoryException, InterruptedExcept break; } if (waitTime >= 32) { - throw new OutOfMemoryException("Unable to allocate sv2 buffer after repeated attempts"); + throw new OutOfMemoryException("Unable to allocate sv2 buffer after repeated attempts"); // TODO } } } @@ -566,15 +584,14 @@ private SelectionVector2 newSV2() throws OutOfMemoryException, InterruptedExcept } private VectorContainer constructHyperBatch(List batchGroupList) { - VectorContainer cont = new VectorContainer(); + final VectorContainer cont = new VectorContainer(); for (MaterializedField field : schema) { - ValueVector[] vectors = new ValueVector[batchGroupList.size()]; + final ValueVector[] vectors = new ValueVector[batchGroupList.size()]; int i = 0; for (BatchGroup group : batchGroupList) { vectors[i++] = group.getValueAccessorById( - field.getValueClass(), - group.getValueVectorId(field.getPath()).getFieldIds()) - .getValueVector(); + field.getValueClass(), group.getValueVectorId(field.getPath()).getFieldIds()) + .getValueVector(); } cont.add(vectors); } @@ -588,33 +605,34 @@ private MSorter createNewMSorter() throws ClassTransformationException, IOExcept private MSorter createNewMSorter(FragmentContext context, List orderings, VectorAccessible batch, MappingSet mainMapping, MappingSet leftMapping, MappingSet rightMapping) throws ClassTransformationException, IOException, SchemaChangeException{ - CodeGenerator cg = CodeGenerator.get(MSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry()); - ClassGenerator g = cg.getRoot(); + final CodeGenerator cg = CodeGenerator.get(MSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry()); + final ClassGenerator g = cg.getRoot(); g.setMappingSet(mainMapping); for (Ordering od : orderings) { // first, we rewrite the evaluation stack for each side of the comparison. - ErrorCollector collector = new ErrorCollectorImpl(); - final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), batch, collector, context.getFunctionRegistry()); + final ErrorCollector collector = new ErrorCollectorImpl(); + final LogicalExpression expr = + ExpressionTreeMaterializer.materialize(od.getExpr(), batch, collector, context.getFunctionRegistry()); if (collector.hasErrors()) { throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString()); } g.setMappingSet(leftMapping); - HoldingContainer left = g.addExpr(expr, false); + final HoldingContainer left = g.addExpr(expr, false); g.setMappingSet(rightMapping); - HoldingContainer right = g.addExpr(expr, false); + final HoldingContainer right = g.addExpr(expr, false); g.setMappingSet(mainMapping); // next we wrap the two comparison sides and add the expression block for the comparison. - LogicalExpression fh = + final LogicalExpression fh = FunctionGenerationHelper.getOrderingComparator(od.nullsSortHigh(), left, right, context.getFunctionRegistry()); - HoldingContainer out = g.addExpr(fh, false); - JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0))); + final HoldingContainer out = g.addExpr(fh, false); + final JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0))); if (od.getDirection() == Direction.ASCENDING) { jc._then()._return(out.getValue()); - }else{ + } else { jc._then()._return(out.getValue().minus()); } g.rotateBlock(); @@ -624,42 +642,42 @@ private MSorter createNewMSorter(FragmentContext context, List orderin g.getEvalBlock()._return(JExpr.lit(0)); return context.getImplementationClass(cg); - - } public SingleBatchSorter createNewSorter(FragmentContext context, VectorAccessible batch) throws ClassTransformationException, IOException, SchemaChangeException{ - CodeGenerator cg = CodeGenerator.get(SingleBatchSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry()); - ClassGenerator g = cg.getRoot(); + final CodeGenerator cg = + CodeGenerator.get(SingleBatchSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry()); + final ClassGenerator g = cg.getRoot(); generateComparisons(g, batch); return context.getImplementationClass(cg); } - private void generateComparisons(ClassGenerator g, VectorAccessible batch) throws SchemaChangeException { + private void generateComparisons(final ClassGenerator g, final VectorAccessible batch) + throws SchemaChangeException { g.setMappingSet(MAIN_MAPPING); for (Ordering od : popConfig.getOrderings()) { // first, we rewrite the evaluation stack for each side of the comparison. - ErrorCollector collector = new ErrorCollectorImpl(); + final ErrorCollector collector = new ErrorCollectorImpl(); final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), batch, collector,context.getFunctionRegistry()); if (collector.hasErrors()) { throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString()); } g.setMappingSet(LEFT_MAPPING); - HoldingContainer left = g.addExpr(expr, false); + final HoldingContainer left = g.addExpr(expr, false); g.setMappingSet(RIGHT_MAPPING); - HoldingContainer right = g.addExpr(expr, false); + final HoldingContainer right = g.addExpr(expr, false); g.setMappingSet(MAIN_MAPPING); // next we wrap the two comparison sides and add the expression block for the comparison. - LogicalExpression fh = + final LogicalExpression fh = FunctionGenerationHelper.getOrderingComparator(od.nullsSortHigh(), left, right, context.getFunctionRegistry()); - HoldingContainer out = g.addExpr(fh, false); - JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0))); + final HoldingContainer out = g.addExpr(fh, false); + final JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0))); if (od.getDirection() == Direction.ASCENDING) { jc._then()._return(out.getValue()); @@ -673,11 +691,13 @@ private void generateComparisons(ClassGenerator g, VectorAccessible batch) throw g.getEvalBlock()._return(JExpr.lit(0)); } - private void createCopier(VectorAccessible batch, List batchGroupList, VectorContainer outputContainer, boolean spilling) throws SchemaChangeException { + private void createCopier(final VectorAccessible batch, final List batchGroupList, + final VectorContainer outputContainer, final boolean spilling) throws SchemaChangeException { try { if (copier == null) { - CodeGenerator cg = CodeGenerator.get(PriorityQueueCopier.TEMPLATE_DEFINITION, context.getFunctionRegistry()); - ClassGenerator g = cg.getRoot(); + final CodeGenerator cg = + CodeGenerator.get(PriorityQueueCopier.TEMPLATE_DEFINITION, context.getFunctionRegistry()); + final ClassGenerator g = cg.getRoot(); generateComparisons(g, batch); @@ -686,12 +706,12 @@ private void createCopier(VectorAccessible batch, List batchGroupLis g.setMappingSet(MAIN_MAPPING); copier = context.getImplementationClass(cg); } else { - copier.cleanup(); + DrillAutoCloseables.closeNoChecked(copier); } - BufferAllocator allocator = spilling ? copierAllocator : oContext.getAllocator(); - for (VectorWrapper i : batch) { - ValueVector v = TypeHelper.getNewVector(i.getField(), allocator); + final BufferAllocator allocator = spilling ? copierAllocator : oContext.getAllocator(); + for (final VectorWrapper i : batch) { + final ValueVector v = TypeHelper.getNewVector(i.getField(), allocator); outputContainer.add(v); } copier.setup(context, allocator, batch, batchGroupList, outputContainer); @@ -702,7 +722,6 @@ private void createCopier(VectorAccessible batch, List batchGroupLis } } - @Override public WritableBatch getWritableBatch() { throw new UnsupportedOperationException("A sort batch is not writable."); @@ -718,16 +737,13 @@ private String getFileName(int spill) { * From the context, get the query id, major fragment id, minor fragment id. This will be used as the file name to * which we will dump the incoming buffer data */ - FragmentHandle handle = context.getHandle(); - - String qid = QueryIdHelper.getQueryId(handle.getQueryId()); - - int majorFragmentId = handle.getMajorFragmentId(); - int minorFragmentId = handle.getMinorFragmentId(); - - String fileName = String.format("%s//%s//major_fragment_%s//minor_fragment_%s//operator_%s//%s", dirs.next(), qid, majorFragmentId, minorFragmentId, popConfig.getOperatorId(), spill); + final FragmentHandle handle = context.getHandle(); + final String qid = QueryIdHelper.getQueryId(handle.getQueryId()); + final int majorFragmentId = handle.getMajorFragmentId(); + final int minorFragmentId = handle.getMinorFragmentId(); + final String fileName = String.format("%s//%s//major_fragment_%s//minor_fragment_%s//operator_%s//%s", + dirs.next(), qid, majorFragmentId, minorFragmentId, popConfig.getOperatorId(), spill); return fileName; } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java index 37529ffb291..d42e8d483d7 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java @@ -37,14 +37,13 @@ import com.google.common.base.Stopwatch; import com.google.common.collect.Queues; -public abstract class MSortTemplate implements MSorter, IndexedSortable{ +public abstract class MSortTemplate implements MSorter, IndexedSortable { // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MSortTemplate.class); private SelectionVector4 vector4; private SelectionVector4 aux; private long compares; private Queue runStarts = Queues.newLinkedBlockingQueue(); - private Queue newRunStarts; private FragmentContext context; /** @@ -67,7 +66,7 @@ public void setup(final FragmentContext context, final BufferAllocator allocator final int newBatch = this.vector4.get(i) >>> 16; if (newBatch == batch) { continue; - } else if(newBatch == batch + 1) { + } else if (newBatch == batch + 1) { runStarts.add(i); batch = newBatch; } else { @@ -135,7 +134,7 @@ public void sort(final VectorContainer container) { } int outIndex = 0; - newRunStarts = Queues.newLinkedBlockingQueue(); + final Queue newRunStarts = Queues.newLinkedBlockingQueue(); newRunStarts.add(outIndex); final int size = runStarts.size(); for (int i = 0; i < size / 2; i++) { @@ -155,9 +154,9 @@ public void sort(final VectorContainer container) { } final SelectionVector4 tmp = aux.createNewWrapperCurrent(desiredRecordBatchCount); aux.clear(); - aux = this.vector4.createNewWrapperCurrent(desiredRecordBatchCount); + aux = vector4.createNewWrapperCurrent(desiredRecordBatchCount); vector4.clear(); - this.vector4 = tmp.createNewWrapperCurrent(desiredRecordBatchCount); + vector4 = tmp.createNewWrapperCurrent(desiredRecordBatchCount); tmp.clear(); runStarts = newRunStarts; } @@ -198,5 +197,4 @@ public void clear() { public abstract void doSetup(@Named("context") FragmentContext context, @Named("incoming") VectorContainer incoming, @Named("outgoing") RecordBatch outgoing); public abstract int doEval(@Named("leftIndex") int leftIndex, @Named("rightIndex") int rightIndex); - } 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 161ca6a26aa..7590546f417 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 @@ -25,15 +25,15 @@ import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.record.VectorAccessible; -public interface PriorityQueueCopier { - public static long initialAllocation = 10000000; - public static long maxAllocation = 20000000; +public interface PriorityQueueCopier extends AutoCloseable { + public static final long INITIAL_ALLOCATION = 10000000; + public static final long MAX_ALLOCATION = 20000000; - public void setup(FragmentContext context, BufferAllocator allocator, VectorAccessible hyperBatch, List batchGroups, - VectorAccessible outgoing) throws SchemaChangeException; - public int next(int targetRecordCount); - public void cleanup(); + public void setup(FragmentContext context, BufferAllocator allocator, VectorAccessible hyperBatch, + List batchGroups, VectorAccessible outgoing) throws SchemaChangeException; - public static TemplateClassDefinition TEMPLATE_DEFINITION = new TemplateClassDefinition(PriorityQueueCopier.class, PriorityQueueCopierTemplate.class); + public int next(int targetRecordCount); + public final static TemplateClassDefinition TEMPLATE_DEFINITION = + new TemplateClassDefinition(PriorityQueueCopier.class, PriorityQueueCopierTemplate.class); } 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 facf1929fc1..f9295c4dbf9 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 @@ -32,7 +32,7 @@ import org.apache.drill.exec.vector.AllocationHelper; public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueCopierTemplate.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueCopierTemplate.class); private SelectionVector4 vector4; private List batchGroups; @@ -42,8 +42,8 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier private int queueSize = 0; @Override - public void setup(FragmentContext context, BufferAllocator allocator, VectorAccessible hyperBatch, List batchGroups, - VectorAccessible outgoing) throws SchemaChangeException { + public void setup(FragmentContext context, BufferAllocator allocator, VectorAccessible hyperBatch, + List batchGroups, VectorAccessible outgoing) throws SchemaChangeException { this.hyperBatch = hyperBatch; this.batchGroups = batchGroups; this.outgoing = outgoing; @@ -66,7 +66,12 @@ public int next(int targetRecordCount) { allocateVectors(targetRecordCount); for (int outgoingIndex = 0; outgoingIndex < targetRecordCount; outgoingIndex++) { if (queueSize == 0) { - cleanup(); + // TODO replace with AutoCloseables.closeNoChecked() when that is merged + try { + close(); + } catch(Exception e) { + throw new RuntimeException("Exception caught from close", e); + } return 0; } int compoundIndex = vector4.get(0); @@ -90,18 +95,18 @@ public int next(int targetRecordCount) { } private void setValueCount(int count) { - for (VectorWrapper w: outgoing) { + for (VectorWrapper w: outgoing) { w.getValueVector().getMutator().setValueCount(count); } } @Override - public void cleanup() { + public void close() throws Exception { vector4.clear(); - for (VectorWrapper w: outgoing) { + for (VectorWrapper w: outgoing) { w.getValueVector().clear(); } - for (VectorWrapper w : hyperBatch) { + for (VectorWrapper w : hyperBatch) { w.clear(); } } @@ -119,7 +124,7 @@ private void siftUp() { } private void allocateVectors(int targetRecordCount) { - for (VectorWrapper w: outgoing) { + for (VectorWrapper w: outgoing) { AllocationHelper.allocateNew(w.getValueVector(), targetRecordCount); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java index daa72766472..5fde86fd05c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java @@ -61,9 +61,55 @@ import com.google.common.collect.ImmutableSet.Builder; public class DrillRuleSets { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRuleSets.class); + // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRuleSets.class); - public static RuleSet DRILL_BASIC_RULES = null; + private final static ImmutableSet staticRuleSet = ImmutableSet.builder().add( + // Add support for Distinct Union (by using Union-All followed by Distinct) + UnionToDistinctRule.INSTANCE, + + // Add support for WHERE style joins. + DrillFilterJoinRules.DRILL_FILTER_ON_JOIN, + DrillFilterJoinRules.DRILL_JOIN, + // End support for WHERE style joins. + + /* + Filter push-down related rules + */ + DrillPushFilterPastProjectRule.INSTANCE, + FilterSetOpTransposeRule.INSTANCE, + + FilterMergeRule.INSTANCE, + AggregateRemoveRule.INSTANCE, + ProjectRemoveRule.NAME_CALC_INSTANCE, + SortRemoveRule.INSTANCE, + + AggregateExpandDistinctAggregatesRule.INSTANCE, + DrillReduceAggregatesRule.INSTANCE, + + /* + Projection push-down related rules + */ + DrillPushProjectPastFilterRule.INSTANCE, + DrillPushProjectPastJoinRule.INSTANCE, + DrillPushProjIntoScan.INSTANCE, + DrillProjectSetOpTransposeRule.INSTANCE, + + /* + Convert from Calcite Logical to Drill Logical Rules. + */ + ExpandConversionRule.INSTANCE, + DrillScanRule.INSTANCE, + DrillFilterRule.INSTANCE, + DrillProjectRule.INSTANCE, + DrillWindowRule.INSTANCE, + DrillAggregateRule.INSTANCE, + + DrillLimitRule.INSTANCE, + DrillSortRule.INSTANCE, + DrillJoinRule.INSTANCE, + DrillUnionAllRule.INSTANCE, + DrillValuesRule.INSTANCE + ).build(); /** * Get a list of logical rules that can be turned on or off by session/system options. @@ -119,65 +165,23 @@ public static RuleSet getDrillUserConfigurableLogicalRules(QueryContext queryCon * Note : Join permutation rule is excluded here. */ public static RuleSet getDrillBasicRules(QueryContext context) { - if (DRILL_BASIC_RULES == null) { - - DRILL_BASIC_RULES = new DrillRuleSet(ImmutableSet. builder().add( // - // Add support for Distinct Union (by using Union-All followed by Distinct) - UnionToDistinctRule.INSTANCE, - - // Add support for WHERE style joins. - DrillFilterJoinRules.DRILL_FILTER_ON_JOIN, - DrillFilterJoinRules.DRILL_JOIN, - // End support for WHERE style joins. - - /* - Filter push-down related rules - */ - DrillPushFilterPastProjectRule.INSTANCE, - FilterSetOpTransposeRule.INSTANCE, - - FilterMergeRule.INSTANCE, - AggregateRemoveRule.INSTANCE, - ProjectRemoveRule.NAME_CALC_INSTANCE, - SortRemoveRule.INSTANCE, - - DrillMergeProjectRule.getInstance(true, RelFactories.DEFAULT_PROJECT_FACTORY, context.getFunctionRegistry()), - AggregateExpandDistinctAggregatesRule.INSTANCE, - DrillReduceAggregatesRule.INSTANCE, - - /* - Projection push-down related rules - */ - DrillPushProjectPastFilterRule.INSTANCE, - DrillPushProjectPastJoinRule.INSTANCE, - DrillPushProjIntoScan.INSTANCE, - DrillProjectSetOpTransposeRule.INSTANCE, - - PruneScanRule.getFilterOnProject(context), - PruneScanRule.getFilterOnScan(context), - PruneScanRule.getFilterOnProjectParquet(context), - PruneScanRule.getFilterOnScanParquet(context), - - /* - Convert from Calcite Logical to Drill Logical Rules. - */ - ExpandConversionRule.INSTANCE, - DrillScanRule.INSTANCE, - DrillFilterRule.INSTANCE, - DrillProjectRule.INSTANCE, - DrillWindowRule.INSTANCE, - DrillAggregateRule.INSTANCE, - - DrillLimitRule.INSTANCE, - DrillSortRule.INSTANCE, - DrillJoinRule.INSTANCE, - DrillUnionAllRule.INSTANCE, - DrillValuesRule.INSTANCE - ) - .build()); - } - - return DRILL_BASIC_RULES; + /* + * We have to create another copy of the ruleset with the context dependent elements; + * this cannot be reused across queries. + */ + final ImmutableSet basicRules = ImmutableSet.builder() + .addAll(staticRuleSet) + .add( + DrillMergeProjectRule.getInstance(true, RelFactories.DEFAULT_PROJECT_FACTORY, context.getFunctionRegistry()), + + PruneScanRule.getFilterOnProject(context), + PruneScanRule.getFilterOnScan(context), + PruneScanRule.getFilterOnProjectParquet(context), + PruneScanRule.getFilterOnScanParquet(context) + ) + .build(); + + return new DrillRuleSet(basicRules); } // Ruleset for join permutation, used only in VolcanoPlanner. @@ -266,7 +270,6 @@ private static class DrillRuleSet implements RuleSet{ final ImmutableSet rules; public DrillRuleSet(ImmutableSet rules) { - super(); this.rules = rules; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRel.java index 58c42fc79e1..b49f846c234 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRel.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRel.java @@ -19,7 +19,6 @@ package org.apache.drill.exec.planner.logical; import com.google.common.collect.Lists; -import com.sun.java.swing.plaf.windows.resources.windows; import org.apache.calcite.linq4j.Ord; import org.apache.calcite.util.BitSets; import org.apache.drill.common.expression.ExpressionPosition; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java index 873173915f9..63ca59bbdc3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java @@ -32,7 +32,7 @@ import org.apache.drill.exec.record.selection.SelectionVector4; public abstract class AbstractRecordBatch implements CloseableRecordBatch { - final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(this.getClass()); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(new Object() {}.getClass().getEnclosingClass()); protected final VectorContainer container; protected final T popConfig; @@ -51,8 +51,7 @@ protected AbstractRecordBatch(final T popConfig, final FragmentContext context, } protected AbstractRecordBatch(final T popConfig, final FragmentContext context, final boolean buildSchema, - final OperatorContext oContext) throws OutOfMemoryException { - super(); + final OperatorContext oContext) { this.context = context; this.popConfig = popConfig; this.oContext = oContext; @@ -119,6 +118,7 @@ public final IterOutcome next(final int inputIndex, final RecordBatch b){ return next; } + @Override public final IterOutcome next() { try { stats.startProcessing(); @@ -174,11 +174,11 @@ public void kill(final boolean sendUpstream) { protected abstract void killIncoming(boolean sendUpstream); - public void close(){ + @Override + public void close() throws Exception { container.clear(); } - @Override public SelectionVector2 getSelectionVector2() { throw new UnsupportedOperationException(); @@ -199,7 +199,6 @@ public VectorWrapper getValueAccessorById(final Class clazz, final int... return container.getValueAccessorById(clazz, ids); } - @Override public WritableBatch getWritableBatch() { // logger.debug("Getting writable batch."); @@ -212,5 +211,4 @@ public WritableBatch getWritableBatch() { public VectorContainer getOutgoingContainer() { throw new UnsupportedOperationException(String.format(" You should not call getOutgoingContainer() for class %s", this.getClass().getCanonicalName())); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java index dd90cab45ae..e3aca45a808 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java @@ -25,7 +25,7 @@ import org.apache.drill.exec.vector.SchemaChangeCallBack; public abstract class AbstractSingleRecordBatch extends AbstractRecordBatch { - final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(this.getClass()); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(new Object() {}.getClass().getEnclosingClass()); protected final RecordBatch incoming; protected boolean outOfMemory = false; @@ -51,7 +51,7 @@ public IterOutcome innerNext() { IterOutcome upstream = next(incoming); if (state != BatchState.FIRST && upstream == IterOutcome.OK && incoming.getRecordCount() == 0) { do { - for (VectorWrapper w : incoming) { + for (VectorWrapper w : incoming) { w.clear(); } } while ((upstream = next(incoming)) == IterOutcome.OK && incoming.getRecordCount() == 0); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java index f2f94509c17..732129a60b1 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java @@ -25,21 +25,19 @@ import org.apache.drill.exec.rpc.data.AckSender; public class RawFragmentBatch { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RawFragmentBatch.class); + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RawFragmentBatch.class); private final FragmentRecordBatch header; private final DrillBuf body; private final AckSender sender; - - private AtomicBoolean ackSent = new AtomicBoolean(false); + private final AtomicBoolean ackSent = new AtomicBoolean(false); public RawFragmentBatch(FragmentRecordBatch header, DrillBuf body, AckSender sender) { - super(); this.header = header; - this.body = body; this.sender = sender; + this.body = body; if (body != null) { - body.retain(); + body.retain(1); } } @@ -58,11 +56,10 @@ public String toString() { public void release() { if (body != null) { - body.release(); + body.release(1); } } - public AckSender getSender() { return sender; } @@ -80,5 +77,4 @@ public long getByteCount() { public boolean isAckSent() { return ackSent.get(); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java index 6f10a1cbbf6..e7e514efd74 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java @@ -17,7 +17,6 @@ */ package org.apache.drill.exec.record; -import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.record.selection.SelectionVector2; import org.apache.drill.exec.record.selection.SelectionVector4; @@ -30,7 +29,7 @@ * A key thing to know is that the Iterator provided by record batch must align with the rank positions of the field ids * provided utilizing getValueVectorId(); */ -public interface RecordBatch extends VectorAccessible { +public interface RecordBatch extends VectorAccessible, AutoCloseable { /* max batch size, limited by 2-byte-length in SV2 : 65536 = 2^16 */ public static final int MAX_BATCH_SIZE = 65536; @@ -58,48 +57,19 @@ public static enum SetupOutcome { * * @return */ - public FragmentContext getContext(); - - /** - * Provide the schema of the current RecordBatch. This changes if and only if a *_NEW_SCHEMA IterOutcome is provided. - * - * @return - */ - public BatchSchema getSchema(); - - /** - * Provide the number of records that are within this record count - * - * @return - */ - public int getRecordCount(); + FragmentContext getContext(); /** * Inform child nodes that this query should be terminated. Child nodes should utilize the QueryContext to determine * what has happened. */ - public void kill(boolean sendUpstream); - - public abstract SelectionVector2 getSelectionVector2(); - - public abstract SelectionVector4 getSelectionVector4(); + void kill(boolean sendUpstream); + SelectionVector2 getSelectionVector2(); - public VectorContainer getOutgoingContainer(); + SelectionVector4 getSelectionVector4(); - /** - * Get the value vector type and id for the given schema path. The TypedFieldId should store a fieldId which is the - * same as the ordinal position of the field within the Iterator provided this classes implementation of - * Iterable. - * - * @param path - * The path where the vector should be located. - * @return The local field id associated with this vector. If no field matches this path, this will return a null - * TypedFieldId - */ - public abstract TypedFieldId getValueVectorId(SchemaPath path); - @Override - public abstract VectorWrapper getValueAccessorById(Class clazz, int... ids); + VectorContainer getOutgoingContainer(); /** * Update the data in each Field reading interface for the next range of records. Once a RecordBatch returns an @@ -108,13 +78,12 @@ public static enum SetupOutcome { * * @return An IterOutcome describing the result of the iteration. */ - public IterOutcome next(); + IterOutcome next(); /** * Get a writable version of this batch. Takes over owernship of existing buffers. * * @return */ - public WritableBatch getWritableBatch(); - + WritableBatch getWritableBatch(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java index 8e3b9e54833..23f7ec300c3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; +import org.apache.drill.common.StackTrace; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.exec.exception.SchemaChangeException; import org.apache.drill.exec.expr.TypeHelper; @@ -34,9 +35,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Maps; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; + public class RecordBatchLoader implements VectorAccessible, Iterable>{ private final static Logger logger = LoggerFactory.getLogger(RecordBatchLoader.class); @@ -63,7 +66,7 @@ public RecordBatchLoader(BufferAllocator allocator) { public boolean load(RecordBatchDef def, DrillBuf buf) throws SchemaChangeException { if (logger.isTraceEnabled()) { logger.trace("Loading record batch with def {} and data {}", def, buf); - logger.trace("Load, ThreadID: {}", Thread.currentThread().getId(), new RuntimeException("For Stack Trace Only")); + logger.trace("Load, ThreadID: {}\n{}", Thread.currentThread().getId(), new StackTrace()); } container.zeroVectors(); valueCount = def.getRecordCount(); @@ -132,12 +135,11 @@ public boolean load(RecordBatchDef def, DrillBuf buf) throws SchemaChangeExcepti return schemaChanged; } + @Override public TypedFieldId getValueVectorId(SchemaPath path) { return container.getValueVectorId(path); } - - // // @SuppressWarnings("unchecked") // public T getValueVectorId(int fieldId, Class clazz) { @@ -152,10 +154,12 @@ public TypedFieldId getValueVectorId(SchemaPath path) { // return (T) v; // } + @Override public int getRecordCount() { return valueCount; } + @Override public VectorWrapper getValueAccessorById(Class clazz, int... ids){ return container.getValueAccessorById(clazz, ids); } @@ -170,11 +174,12 @@ public Iterator> iterator() { return this.container.iterator(); } - public BatchSchema getSchema(){ + @Override + public BatchSchema getSchema() { return schema; } - public void clear(){ + public void clear() { container.clear(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorAccessible.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorAccessible.java index 9db16817ab0..6eb58c53e9a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorAccessible.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorAccessible.java @@ -19,9 +19,35 @@ import org.apache.drill.common.expression.SchemaPath; +// TODO javadoc public interface VectorAccessible extends Iterable> { + // TODO are these releated in any way? Should they be the same one? + // TODO javadoc public VectorWrapper getValueAccessorById(Class clazz, int... fieldIds); + + /** + * Get the value vector type and id for the given schema path. The TypedFieldId + * should store a fieldId which is the same as the ordinal position of the field + * within the Iterator provided this classes implementation of Iterable. + * + * @param path the path where the vector should be located. + * @return the local field id associated with this vector. If no field matches this + * path, this will return a null TypedFieldId + */ public TypedFieldId getValueVectorId(SchemaPath path); + + /** + * Get the schema of the current RecordBatch. This changes if and only if a *_NEW_SCHEMA + * IterOutcome is provided. + * + * @return schema of the current batch + */ public BatchSchema getSchema(); + + /** + * Get the number of records. + * + * @return number of records + */ public int getRecordCount(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java index 324829ab4b5..b3726aa68f6 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java @@ -30,10 +30,11 @@ import com.google.common.collect.Lists; /** - * A specialized version of record batch that can moves out buffers and preps them for writing. + * A specialized version of record batch that can move out buffers and preps + * them for writing. */ -public class WritableBatch { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WritableBatch.class); +public class WritableBatch implements AutoCloseable { + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WritableBatch.class); private final RecordBatchDef def; private final DrillBuf[] buffers; @@ -45,7 +46,6 @@ private WritableBatch(RecordBatchDef def, List buffers) { } private WritableBatch(RecordBatchDef def, DrillBuf[] buffers) { - super(); this.def = def; this.buffers = buffers; } @@ -61,23 +61,24 @@ public DrillBuf[] getBuffers() { public void reconstructContainer(VectorContainer container) { Preconditions.checkState(!cleared, "Attempted to reconstruct a container from a WritableBatch after it had been cleared"); - if (buffers.length > 0) { /* If we have DrillBuf's associated with value vectors */ + // If we have DrillBuf's associated with value vectors. + if (buffers.length > 0) { int len = 0; - for (DrillBuf b : buffers) { + for (final DrillBuf b : buffers) { len += b.capacity(); } - DrillBuf newBuf = buffers[0].getAllocator().buffer(len); + final DrillBuf newBuf = buffers[0].getAllocator().buffer(len); try { /* Copy data from each buffer into the compound buffer */ int offset = 0; - for (DrillBuf buf : buffers) { + for (final DrillBuf buf : buffers) { newBuf.setBytes(offset, buf); offset += buf.capacity(); - buf.release(); + buf.release(1); } - List fields = def.getFieldList(); + final List fields = def.getFieldList(); int bufferOffset = 0; @@ -86,11 +87,10 @@ public void reconstructContainer(VectorContainer container) { */ int vectorIndex = 0; - for (VectorWrapper vv : container) { - SerializedField fmd = fields.get(vectorIndex); - ValueVector v = vv.getValueVector(); - DrillBuf bb = newBuf.slice(bufferOffset, fmd.getBufferLength()); -// v.load(fmd, cbb.slice(bufferOffset, fmd.getBufferLength())); + for (final VectorWrapper vv : container) { + final SerializedField fmd = fields.get(vectorIndex); + final ValueVector v = vv.getValueVector(); + final DrillBuf bb = newBuf.slice(bufferOffset, fmd.getBufferLength()); v.load(fmd, bb); vectorIndex++; bufferOffset += fmd.getBufferLength(); @@ -101,8 +101,9 @@ public void reconstructContainer(VectorContainer container) { } } - SelectionVectorMode svMode; - if (def.hasCarriesTwoByteSelectionVector() && def.getCarriesTwoByteSelectionVector()) { + final SelectionVectorMode svMode; + if (def.hasCarriesTwoByteSelectionVector() + && def.getCarriesTwoByteSelectionVector()) { svMode = SelectionVectorMode.TWO_BYTE; } else { svMode = SelectionVectorMode.NONE; @@ -110,76 +111,87 @@ public void reconstructContainer(VectorContainer container) { container.buildSchema(svMode); /* Set the record count in the value vector */ - for (VectorWrapper v : container) { - ValueVector.Mutator m = v.getValueVector().getMutator(); + for (final VectorWrapper v : container) { + final ValueVector.Mutator m = v.getValueVector().getMutator(); m.setValueCount(def.getRecordCount()); } } public void clear() { - if(cleared) { + if (cleared) { return; } - for (DrillBuf buf : buffers) { - buf.release(); + for (final DrillBuf buf : buffers) { + buf.release(1); } cleared = true; } - public static WritableBatch getBatchNoHVWrap(int recordCount, Iterable> vws, boolean isSV2) { - List vectors = Lists.newArrayList(); - for (VectorWrapper vw : vws) { + // TODO need to indicate that the underlying buffers are adopted + public static WritableBatch getBatchNoHVWrap(int recordCount, + Iterable> vws, boolean isSV2) { + final List vectors = Lists.newArrayList(); + for (final VectorWrapper vw : vws) { Preconditions.checkArgument(!vw.isHyper()); vectors.add(vw.getValueVector()); } return getBatchNoHV(recordCount, vectors, isSV2); } - public static WritableBatch getBatchNoHV(int recordCount, Iterable vectors, boolean isSV2) { - List buffers = Lists.newArrayList(); - List metadata = Lists.newArrayList(); + // TODO need to indicate that the underlying buffers are adopted + public static WritableBatch getBatchNoHV(int recordCount, + Iterable vectors, boolean isSV2) { + final List buffers = Lists.newArrayList(); + final List metadata = Lists.newArrayList(); - for (ValueVector vv : vectors) { + for (final ValueVector vv : vectors) { metadata.add(vv.getMetadata()); - // don't try to get the buffers if we don't have any records. It is possible the buffers are dead buffers. + // don't try to get the buffers if we don't have any records. It is + // possible the buffers are dead buffers. if (recordCount == 0) { vv.clear(); continue; } - for (DrillBuf b : vv.getBuffers(true)) { + for (final DrillBuf b : vv.getBuffers(true)) { + assert b.refCnt() > 0 : "value vector buffer is unreferenced"; buffers.add(b); } - // remove vv access to buffers. - vv.clear(); } - RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount) - .setCarriesTwoByteSelectionVector(isSV2).build(); - WritableBatch b = new WritableBatch(batchDef, buffers); + final RecordBatchDef batchDef = RecordBatchDef.newBuilder() + .addAllField(metadata) + .setRecordCount(recordCount) + .setCarriesTwoByteSelectionVector(isSV2) + .build(); + final WritableBatch b = new WritableBatch(batchDef, buffers); return b; } + // TODO need to indicate that the underlying buffers are adopted public static WritableBatch get(RecordBatch batch) { - if (batch.getSchema() != null && batch.getSchema().getSelectionVectorMode() == SelectionVectorMode.FOUR_BYTE) { - throw new UnsupportedOperationException("Only batches without hyper selections vectors are writable."); + if (batch.getSchema() != null + && batch.getSchema().getSelectionVectorMode() == SelectionVectorMode.FOUR_BYTE) { + throw new UnsupportedOperationException( + "Only batches without hyper selections vectors are writable."); } - boolean sv2 = (batch.getSchema().getSelectionVectorMode() == SelectionVectorMode.TWO_BYTE); + final boolean sv2 = (batch.getSchema().getSelectionVectorMode() == SelectionVectorMode.TWO_BYTE); return getBatchNoHVWrap(batch.getRecordCount(), batch, sv2); } - public void retainBuffers() { - for (DrillBuf buf : buffers) { - buf.retain(); + public void retainBuffers(final int increment) { + assert increment > 0 : "retain increment must be positive"; + for (final DrillBuf buf : buffers) { + buf.retain(increment); } } - public void retainBuffers(int increment) { - for (DrillBuf buf : buffers) { - buf.retain(increment); + @Override + public void close() { + for(final DrillBuf drillBuf : buffers) { + drillBuf.release(1); } } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java index ba8640a7804..8bf09f1ccd4 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java @@ -29,8 +29,8 @@ /** * A selection vector that fronts, at most, a */ -public class SelectionVector2 implements Closeable{ -// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector2.class); +public class SelectionVector2 implements AutoCloseable { + // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector2.class); private final BufferAllocator allocator; private int recordCount; @@ -42,7 +42,7 @@ public SelectionVector2(BufferAllocator allocator) { this.allocator = allocator; } - public int getCount(){ + public int getCount() { return recordCount; } @@ -55,7 +55,7 @@ public DrillBuf getBuffer(boolean clear) { if (clear) { /* Increment the ref count for this buffer */ - bufferHandle.retain(); + bufferHandle.retain(1); /* We are passing ownership of the buffer to the * caller. clear the buffer from within our selection vector @@ -66,28 +66,27 @@ public DrillBuf getBuffer(boolean clear) { return bufferHandle; } - public void setBuffer(DrillBuf bufferHandle) - { + public void setBuffer(DrillBuf bufferHandle) { /* clear the existing buffer */ clear(); this.buffer = bufferHandle; - buffer.retain(); + buffer.retain(1); } - public char getIndex(int index){ + public char getIndex(int index) { return buffer.getChar(index * RECORD_SIZE); } - public void setIndex(int index, char value){ + public void setIndex(int index, char value) { buffer.setChar(index * RECORD_SIZE, value); } - public long getDataAddr(){ + public long getDataAddr() { return buffer.memoryAddress(); } - public void setIndex(int index, int value){ + public void setIndex(int index, int value) { buffer.setChar(index, value); } @@ -106,7 +105,7 @@ public void allocateNew(int size) { } @Override - public SelectionVector2 clone(){ + public SelectionVector2 clone() { SelectionVector2 newSV = new SelectionVector2(allocator); newSV.recordCount = recordCount; newSV.buffer = buffer; @@ -115,7 +114,7 @@ public SelectionVector2 clone(){ * same buffer, if we don't do a retain() on the newSV's * buffer, it might get freed. */ - newSV.buffer.retain(); + newSV.buffer.retain(1); clear(); return newSV; } @@ -134,9 +133,7 @@ public void setRecordCount(int recordCount){ } @Override - public void close() throws IOException { + public void close() { clear(); } - - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java index 8db043780d4..3b8dd0d1314 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java @@ -22,8 +22,8 @@ import org.apache.drill.exec.exception.SchemaChangeException; import org.apache.drill.exec.record.DeadBuf; -public class SelectionVector4 { -// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector4.class); +public class SelectionVector4 implements AutoCloseable { + // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector4.class); private ByteBuf data; private int recordCount; @@ -73,7 +73,7 @@ public int get(int index) { public SelectionVector4 createNewWrapperCurrent(int batchRecordCount) { try { data.retain(); - SelectionVector4 sv4 = new SelectionVector4(data, recordCount, batchRecordCount); + final SelectionVector4 sv4 = new SelectionVector4(data, recordCount, batchRecordCount); sv4.start = this.start; return sv4; } catch (SchemaChangeException e) { @@ -116,4 +116,8 @@ public void clear() { } } + @Override + public void close() { + clear(); + } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java index 74a4afb18a9..ac48187627d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java @@ -70,7 +70,7 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf buffer, List ou final int pBodyLength = readRawVarint32(is); final ByteBuf pBody = buffer.slice(buffer.readerIndex(), pBodyLength); buffer.skipBytes(pBodyLength); - pBody.retain(); + pBody.retain(1); if (RpcConstants.EXTRA_DEBUGGING) { logger.debug("Read protobuf body of length {} into buffer {}.", pBodyLength, pBody); } @@ -94,7 +94,7 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf buffer, List ou throw new CorruptedFrameException(String.format("Expected to receive a raw body of %d bytes but received a buffer with %d bytes.", dBodyLength, buffer.readableBytes())); } dBody = buffer.slice(); - dBody.retain(); + dBody.retain(1); if (RpcConstants.EXTRA_DEBUGGING) { logger.debug("Read raw body of {}", dBody); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java index 4908c188837..7e630797d95 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java @@ -26,6 +26,7 @@ import java.io.IOException; import org.apache.drill.exec.exception.FragmentSetupException; +import org.apache.drill.exec.memory.AllocatorClosedException; import org.apache.drill.exec.memory.BufferAllocator; import org.apache.drill.exec.proto.BitData.BitClientHandshake; import org.apache.drill.exec.proto.BitData.BitServerHandshake; @@ -169,7 +170,25 @@ private void send(final FragmentRecordBatch fragmentBatch, final DrillBuf body, final boolean withinMemoryEnvelope; - withinMemoryEnvelope = allocator.takeOwnership(body, out); + try { + withinMemoryEnvelope = allocator.shareOwnership((DrillBuf) body, out); + } catch(final AllocatorClosedException e) { + /* + * It can happen that between the time we get the fragment manager and we + * try to transfer this buffer to it, the fragment may have been cancelled + * and closed. When that happens, the allocator will be closed when we + * attempt this. That just means we can drop this data on the floor, since + * the receiver no longer exists (and no longer needs it). + * + * Note that checking manager.isCancelled() before we attempt this isn't enough, + * because of timing: it may still be cancelled between that check and + * the attempt to do the memory transfer. To double check ourselves, we + * do check manager.isCancelled() here, after the fact; it shouldn't + * change again after its allocator has been closed. + */ + assert manager.isCancelled(); + return; + } if (!withinMemoryEnvelope) { // if we over reserved, we need to add poison pill before batch. @@ -185,7 +204,6 @@ private void send(final FragmentRecordBatch fragmentBatch, final DrillBuf body, } private class ProxyCloseHandler implements GenericFutureListener { - private volatile GenericFutureListener handler; public ProxyCloseHandler(GenericFutureListener handler) { @@ -197,7 +215,6 @@ public ProxyCloseHandler(GenericFutureListener handler) { public void operationComplete(ChannelFuture future) throws Exception { handler.operationComplete(future); } - } @Override @@ -214,5 +231,4 @@ public void handle() { public ProtobufLengthDecoder getDecoder(BufferAllocator allocator, OutOfMemoryHandler outOfMemoryHandler) { return new DataProtobufLengthDecoder.Server(allocator, outOfMemoryHandler); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryDataBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryDataBatch.java index 914bd00b360..f2ef4143e29 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryDataBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryDataBatch.java @@ -22,17 +22,17 @@ import org.apache.drill.exec.proto.UserBitShared.QueryData; public class QueryDataBatch { -// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryResultBatch.class); + // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryDataBatch.class); private final QueryData header; private final DrillBuf data; public QueryDataBatch(QueryData header, DrillBuf data) { -// logger.debug("New Result Batch with header {} and data {}", header, data); + // logger.debug("New Result Batch with header {} and data {}", header, data); this.header = header; this.data = data; if (this.data != null) { - data.retain(); + data.retain(1); } } @@ -50,7 +50,7 @@ public boolean hasData() { public void release() { if (data != null) { - data.release(); + data.release(1); } } @@ -58,5 +58,4 @@ public void release() { public String toString() { return "QueryResultBatch [header=" + header + ", data=" + data + "]"; } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java index 41bb4132680..14c7154e188 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java @@ -148,17 +148,17 @@ public void batchArrived( ConnectionThrottle throttle, ByteBuf pBody, ByteBuf dBody ) throws RpcException { final QueryData queryData = RpcBus.get( pBody, QueryData.PARSER ); // Current batch coming in. - final QueryDataBatch batch = new QueryDataBatch( queryData, (DrillBuf) dBody ); + final DrillBuf drillBuf = (DrillBuf) dBody; + final QueryDataBatch batch = new QueryDataBatch( queryData, drillBuf ); final QueryId queryId = queryData.getQueryId(); logger.debug( "batchArrived: queryId = {}", queryId ); logger.trace( "batchArrived: batch = {}", batch ); - UserResultsListener resultsListener = newUserResultsListener(queryId); + final UserResultsListener resultsListener = newUserResultsListener(queryId); // A data case--pass on via dataArrived - try { resultsListener.dataArrived(batch, throttle); // That releases batch if successful. diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java index b39a10341a1..dde3e49ec92 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java @@ -48,7 +48,6 @@ public class UserClient extends BasicClientWithConnection handler, DrillbitEndpoint endpoint, UserProperties props, UserBitShared.UserCredentials credentials) - throws RpcException { + public void connect(RpcConnectionHandler handler, DrillbitEndpoint endpoint, + UserProperties props, UserBitShared.UserCredentials credentials) { UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) @@ -113,7 +112,6 @@ protected Response handleReponse(ConnectionThrottle throttle, int rpcType, ByteB default: throw new RpcException(String.format("Unknown Rpc Type %d. ", rpcType)); } - } @Override @@ -135,5 +133,4 @@ protected void finalizeConnection(BitToUserHandshake handshake, BasicClientWithC public ProtobufLengthDecoder getDecoder(BufferAllocator allocator) { return new UserProtobufLengthDecoder(allocator, OutOfMemoryHandler.DEFAULT_INSTANCE); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java index d0a998ed21a..da6a7420c2a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java @@ -19,20 +19,19 @@ import io.netty.channel.EventLoopGroup; -import java.io.Closeable; - +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.metrics.DrillMetrics; import org.apache.drill.exec.rpc.TransportCheck; import com.codahale.metrics.MetricRegistry; // TODO: Doc. What kind of context? (For what aspects, RPC? What kind of data?) -public class BootStrapContext implements Closeable{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BootStrapContext.class); +public class BootStrapContext implements AutoCloseable { +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BootStrapContext.class); private final DrillConfig config; private final EventLoopGroup loop; @@ -41,12 +40,15 @@ public class BootStrapContext implements Closeable{ private final BufferAllocator allocator; public BootStrapContext(DrillConfig config) { - super(); this.config = config; - this.loop = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), "BitServer-"); - this.loop2 = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), "BitClient-"); - this.metrics = DrillMetrics.getInstance(); - this.allocator = new TopLevelAllocator(config); + loop = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), "BitServer-"); + loop2 = TransportCheck.createEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), "BitClient-"); + metrics = DrillMetrics.getInstance(); + try { + allocator = new RootAllocator(config); + } catch(final Exception e) { + throw new RuntimeException("Couldn't create root allocator", e); + } } public DrillConfig getConfig() { @@ -69,10 +71,10 @@ public BufferAllocator getAllocator() { return allocator; } + @Override public void close() { DrillMetrics.resetMetrics(); loop.shutdownGracefully(); - allocator.close(); + DrillAutoCloseables.closeNoChecked(allocator); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java index 0640dbbc8aa..1a8fb74071c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java @@ -19,7 +19,7 @@ import java.util.concurrent.atomic.AtomicInteger; -import org.apache.drill.common.AutoCloseables; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.StackTrace; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.ExecConstants; @@ -285,11 +285,12 @@ public synchronized void close() { } } + // TODO these should use a DeferredException Closeables.closeQuietly(engine); - AutoCloseables.close(storeProvider, logger); + DrillAutoCloseables.closeNoChecked(storeProvider); Closeables.closeQuietly(coord); - AutoCloseables.close(manager, logger); - Closeables.closeQuietly(context); + DrillAutoCloseables.closeNoChecked(manager); + DrillAutoCloseables.closeNoChecked(context); logger.info("Shutdown completed ({} ms).", System.currentTimeMillis() - startTime); isClosed = true; @@ -326,7 +327,12 @@ public ShutdownThread(final Drillbit drillbit, final StackTrace stackTrace) { public void run() { logger.info("Received shutdown request."); try { - synchronized (idCounter) { + /* + * We can avoid metrics deregistration concurrency issues by only closing + * one drillbit at a time. To enforce that, we synchronize on a convenient + * singleton object. + */ + synchronized(idCounter) { drillbit.close(); } } catch(final Exception e) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java index 6e276280fee..8ca3ec83036 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java @@ -80,7 +80,7 @@ public boolean apply(SchemaPath path) { @Override public void allocate(Map vectorMap) throws OutOfMemoryException { - for (ValueVector v : vectorMap.values()) { + for (final ValueVector v : vectorMap.values()) { v.allocateNew(); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java index 61ccac5f344..c2ab0d04e6b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java @@ -26,30 +26,27 @@ import org.apache.drill.exec.record.MaterializedField.Key; import org.apache.drill.exec.vector.ValueVector; -public interface RecordReader { - +public interface RecordReader extends AutoCloseable { public static final long ALLOCATOR_INITIAL_RESERVATION = 1*1024*1024; public static final long ALLOCATOR_MAX_RESERVATION = 20L*1000*1000*1000; /** * Configure the RecordReader with the provided schema and the record batch that should be written to. * + * @param context operator context for the reader * @param output * The place where output for a particular scan should be written. The record reader is responsible for * mutating the set of schema values for that particular record. * @throws ExecutionSetupException */ - public abstract void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException; + void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException; - public abstract void allocate(Map vectorMap) throws OutOfMemoryException; + void allocate(Map vectorMap) throws OutOfMemoryException; /** * Increment record reader forward, writing into the provided output batch. * * @return The number of additional records added to the output. */ - public abstract int next(); - - public abstract void cleanup(); - -} \ No newline at end of file + int next(); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java index a52fd222bfe..331af10afd9 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java @@ -300,7 +300,7 @@ private void ensure(final int length) { } @Override - public void cleanup() { + public void close() { if (reader != null) { try { reader.close(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java index dfc4f3af9b3..66b0146b2a0 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java @@ -57,7 +57,6 @@ public class JSONRecordReader extends AbstractRecordReader { private int recordCount; private long runningRecordCount = 0; private final FragmentContext fragmentContext; - private OperatorContext operatorContext; private final boolean enableAllTextMode; private final boolean readNumbersAsDouble; @@ -82,13 +81,14 @@ public JSONRecordReader(final FragmentContext fragmentContext, final String inpu * @param columns * @throws OutOfMemoryException */ - public JSONRecordReader(final FragmentContext fragmentContext, final JsonNode embeddedContent, final DrillFileSystem fileSystem, - final List columns) throws OutOfMemoryException { + public JSONRecordReader(final FragmentContext fragmentContext, final JsonNode embeddedContent, + final DrillFileSystem fileSystem, final List columns) throws OutOfMemoryException { this(fragmentContext, null, embeddedContent, fileSystem, columns); } - private JSONRecordReader(final FragmentContext fragmentContext, final String inputPath, final JsonNode embeddedContent, final DrillFileSystem fileSystem, - final List columns) throws OutOfMemoryException { + private JSONRecordReader(final FragmentContext fragmentContext, final String inputPath, + final JsonNode embeddedContent, final DrillFileSystem fileSystem, + final List columns) { Preconditions.checkArgument( (inputPath == null && embeddedContent != null) || @@ -96,9 +96,9 @@ private JSONRecordReader(final FragmentContext fragmentContext, final String inp "One of inputPath or embeddedContent must be set but not both." ); - if(inputPath != null){ + if(inputPath != null) { this.hadoopPath = new Path(inputPath); - }else{ + } else { this.embeddedContent = embeddedContent; } @@ -113,7 +113,6 @@ private JSONRecordReader(final FragmentContext fragmentContext, final String inp @Override public void setup(final OperatorContext context, final OutputMutator output) throws ExecutionSetupException { - this.operatorContext = context; try{ if (hadoopPath != null) { this.stream = fileSystem.openPossiblyCompressedStream(hadoopPath); @@ -131,7 +130,7 @@ public void setup(final OperatorContext context, final OutputMutator output) thr } } - private void setupParser() throws IOException{ + private void setupParser() throws IOException { if(hadoopPath != null){ jsonReader.setSource(stream); }else{ @@ -174,11 +173,11 @@ public int next() { ReadState write = null; // Stopwatch p = new Stopwatch().start(); try{ - outside: while(recordCount < BaseValueVector.INITIAL_VALUE_ALLOCATION){ + outside: while(recordCount < BaseValueVector.INITIAL_VALUE_ALLOCATION) { writer.setPosition(recordCount); write = jsonReader.write(writer); - if(write == ReadState.WRITE_SUCCEED){ + if(write == ReadState.WRITE_SUCCEED) { // logger.debug("Wrote record."); recordCount++; }else{ @@ -195,7 +194,6 @@ public int next() { // System.out.println(String.format("Wrote %d records in %dms.", recordCount, p.elapsed(TimeUnit.MILLISECONDS))); updateRunningCount(); - return recordCount; } catch (final Exception e) { @@ -210,14 +208,9 @@ private void updateRunningCount() { } @Override - public void cleanup() { - try { - if(stream != null){ - stream.close(); - } - } catch (final IOException e) { - logger.warn("Failure while closing stream.", e); + public void close() throws Exception { + if(stream != null) { + stream.close(); } } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonProcessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonProcessor.java index 4d8d4ba0f0c..ccb3c28a67a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonProcessor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonProcessor.java @@ -21,7 +21,6 @@ import java.io.InputStream; import org.apache.drill.common.exceptions.UserException; -import org.apache.drill.exec.proto.UserBitShared; import org.apache.drill.exec.vector.complex.writer.BaseWriter; import com.fasterxml.jackson.databind.JsonNode; @@ -49,5 +48,4 @@ public UserException.Builder getExceptionWithContext(Throwable exception, String field, String msg, Object... args); - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java index 78336315259..a89fa86d571 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java @@ -33,7 +33,7 @@ public abstract class BaseJsonProcessor implements JsonProcessor { - private static final ObjectMapper MAPPER = new ObjectMapper() // + private static final ObjectMapper MAPPER = new ObjectMapper() .configure(JsonParser.Feature.ALLOW_COMMENTS, true) .configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true); @@ -41,12 +41,12 @@ public abstract class BaseJsonProcessor implements JsonProcessor { protected DrillBuf workBuf; public BaseJsonProcessor(DrillBuf workBuf) { - this.workBuf = Preconditions.checkNotNull(workBuf); + workBuf = Preconditions.checkNotNull(workBuf); } @Override public void setSource(InputStream is) throws IOException { - this.parser = MAPPER.getFactory().createParser(is); + parser = MAPPER.getFactory().createParser(is); } @Override @@ -59,10 +59,10 @@ public UserException.Builder getExceptionWithContext(UserException.Builder excep String field, String msg, Object... args) { - if (msg != null){ + if (msg != null) { exceptionBuilder.message(msg, args); } - if(field!=null) { + if(field != null) { exceptionBuilder.pushContext("Field ", field); } exceptionBuilder.pushContext("Column ", parser.getCurrentLocation().getColumnNr()+1) @@ -78,5 +78,4 @@ public UserException.Builder getExceptionWithContext(Throwable e, UserException.Builder exceptionBuilder = UserException.dataReadError(e); return getExceptionWithContext(exceptionBuilder, field, msg, args); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/CompliantTextRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/CompliantTextRecordReader.java index ae11ba7abf3..ad65a9403ee 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/CompliantTextRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/CompliantTextRecordReader.java @@ -144,7 +144,7 @@ public int next() { * This would internally close the input stream we are reading from. */ @Override - public void cleanup() { + public void close() { try { if (reader != null) { reader.close(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java index fd97c484c2d..c742690147d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java @@ -24,7 +24,6 @@ import org.apache.drill.common.types.TypeProtos.MajorType; import org.apache.drill.exec.exception.SchemaChangeException; import org.apache.drill.exec.expr.TypeHelper; -import org.apache.drill.exec.memory.BufferAllocator; import org.apache.drill.exec.memory.OutOfMemoryException; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.ops.OperatorContext; @@ -38,23 +37,19 @@ import org.apache.drill.exec.vector.ValueVector; public class MockRecordReader extends AbstractRecordReader { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class); - private OutputMutator output; - private MockScanEntry config; - private FragmentContext context; - private BufferAllocator alcator; + private final MockScanEntry config; + private final FragmentContext context; private ValueVector[] valueVectors; private int recordsRead; private int batchRecordCount; - private FragmentContext fragmentContext; private OperatorContext operatorContext; - public MockRecordReader(FragmentContext context, MockScanEntry config) throws OutOfMemoryException { + public MockRecordReader(FragmentContext context, MockScanEntry config) { this.context = context; this.config = config; - this.fragmentContext=context; } private int getEstimatedRecordSize(MockColumn[] types) { @@ -67,38 +62,26 @@ private int getEstimatedRecordSize(MockColumn[] types) { private MaterializedField getVector(String name, MajorType type, int length) { assert context != null : "Context shouldn't be null."; - MaterializedField f = MaterializedField.create(SchemaPath.getSimplePath(name), type); - + final MaterializedField f = MaterializedField.create(SchemaPath.getSimplePath(name), type); return f; - - } - - public OperatorContext getOperatorContext() { - return operatorContext; - } - - public void setOperatorContext(OperatorContext operatorContext) { - this.operatorContext = operatorContext; } @Override public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException { try { - this.output = output; - int estimateRowSize = getEstimatedRecordSize(config.getTypes()); + final int estimateRowSize = getEstimatedRecordSize(config.getTypes()); valueVectors = new ValueVector[config.getTypes().length]; batchRecordCount = 250000 / estimateRowSize; for (int i = 0; i < config.getTypes().length; i++) { - MajorType type = config.getTypes()[i].getMajorType(); - MaterializedField field = getVector(config.getTypes()[i].getName(), type, batchRecordCount); - Class vvClass = TypeHelper.getValueVectorClass(field.getType().getMinorType(), field.getDataMode()); + final MajorType type = config.getTypes()[i].getMajorType(); + final MaterializedField field = getVector(config.getTypes()[i].getName(), type, batchRecordCount); + final Class vvClass = TypeHelper.getValueVectorClass(field.getType().getMinorType(), field.getDataMode()); valueVectors[i] = output.addField(field, vvClass); } } catch (SchemaChangeException e) { throw new ExecutionSetupException("Failure while setting up fields", e); } - } @Override @@ -107,23 +90,20 @@ public int next() { return 0; } - int recordSetSize = Math.min(batchRecordCount, this.config.getRecords() - recordsRead); - + final int recordSetSize = Math.min(batchRecordCount, this.config.getRecords() - recordsRead); recordsRead += recordSetSize; - for (ValueVector v : valueVectors) { - -// logger.debug(String.format("MockRecordReader: Generating %d records of random data for VV of type %s.", recordSetSize, v.getClass().getName())); - ValueVector.Mutator m = v.getMutator(); + for (final ValueVector v : valueVectors) { + final ValueVector.Mutator m = v.getMutator(); m.generateTestData(recordSetSize); - } + return recordSetSize; } @Override public void allocate(Map vectorMap) throws OutOfMemoryException { try { - for (ValueVector v : vectorMap.values()) { + for (final ValueVector v : vectorMap.values()) { AllocationHelper.allocate(v, Character.MAX_VALUE, 50, 10); } } catch (NullPointerException e) { @@ -132,7 +112,6 @@ public void allocate(Map vectorMap) throws OutOfMemoryExceptio } @Override - public void cleanup() { + public void close() { } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java index 74423bfda55..3838ffa7afb 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java @@ -30,16 +30,16 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -public class MockScanBatchCreator implements BatchCreator{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class); +public class MockScanBatchCreator implements BatchCreator { + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class); @Override public ScanBatch getBatch(FragmentContext context, MockSubScanPOP config, List children) throws ExecutionSetupException { Preconditions.checkArgument(children.isEmpty()); - List entries = config.getReadEntries(); - List readers = Lists.newArrayList(); - for(MockScanEntry e : entries){ + final List entries = config.getReadEntries(); + final List readers = Lists.newArrayList(); + for(MockScanEntry e : entries) { readers.add(new MockRecordReader(context, e)); } return new ScanBatch(config, context, readers.iterator()); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java index da6fbfbf57b..a4f5cac017b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java @@ -67,9 +67,7 @@ public class ParquetRecordReader extends AbstractRecordReader { private static final char DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH = 32*1024; // TODO - should probably find a smarter way to set this, currently 1 megabyte - private static final int VAR_LEN_FIELD_LENGTH = 1024 * 1024 * 1; public static final int PARQUET_PAGE_MAX_SIZE = 1024 * 1024 * 1; - private static final String SEPERATOR = System.getProperty("file.separator"); // used for clearing the last n bits of a byte public static final byte[] endBitMasks = {-2, -4, -8, -16, -32, -64, -128}; @@ -79,16 +77,16 @@ public class ParquetRecordReader extends AbstractRecordReader { private int bitWidthAllFixedFields; private boolean allFieldsFixedLength; private int recordsPerBatch; + private OperatorContext operatorContext; // private long totalRecords; // private long rowGroupOffset; - private List columnStatuses; + private List> columnStatuses; private FileSystem fileSystem; private long batchSize; Path hadoopPath; private VarLenBinaryReader varLengthReader; private ParquetMetadata footer; - private OperatorContext operatorContext; // This is a parallel list to the columns list above, it is used to determine the subset of the project // pushdown columns that do not appear in this file private boolean[] columnsFound; @@ -160,14 +158,6 @@ public long getBatchSize() { return batchSize; } - public OperatorContext getOperatorContext() { - return operatorContext; - } - - public void setOperatorContext(OperatorContext operatorContext) { - this.operatorContext = operatorContext; - } - /** * @param type a fixed length type from the parquet library enum * @return the length in pageDataByteArray of the type @@ -205,9 +195,13 @@ private boolean fieldSelected(MaterializedField field) { return false; } + public OperatorContext getOperatorContext() { + return operatorContext; + } + @Override - public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException { - this.operatorContext = context; + public void setup(OperatorContext operatorContext, OutputMutator output) throws ExecutionSetupException { + this.operatorContext = operatorContext; if (!isStarQuery()) { columnsFound = new boolean[getColumns().size()]; nullFilledVectors = new ArrayList<>(); @@ -276,7 +270,7 @@ public void setup(OperatorContext context, OutputMutator output) throws Executio try { ValueVector vector; SchemaElement schemaElement; - ArrayList varLengthColumns = new ArrayList<>(); + final ArrayList varLengthColumns = new ArrayList<>(); // initialize all of the column read status objects boolean fieldFixedLength; for (int i = 0; i < columns.size(); ++i) { @@ -342,7 +336,7 @@ protected void handleAndRaise(String s, Exception e) { @Override public void allocate(Map vectorMap) throws OutOfMemoryException { try { - for (ValueVector v : vectorMap.values()) { + for (final ValueVector v : vectorMap.values()) { AllocationHelper.allocate(v, recordsPerBatch, 50, 10); } } catch (NullPointerException e) { @@ -366,17 +360,17 @@ private TypeProtos.DataMode getDataMode(ColumnDescriptor column) { } private void resetBatch() { - for (ColumnReader column : columnStatuses) { + for (final ColumnReader column : columnStatuses) { column.valuesReadInCurrentPass = 0; } - for (VarLengthColumn r : varLengthReader.columns) { + for (final VarLengthColumn r : varLengthReader.columns) { r.valuesReadInCurrentPass = 0; } } public void readAllFixedFields(long recordsToRead) throws IOException { - for (ColumnReader crs : columnStatuses) { + for (ColumnReader crs : columnStatuses) { crs.processPages(recordsToRead); } } @@ -386,7 +380,7 @@ public int next() { resetBatch(); long recordsToRead = 0; try { - ColumnReader firstColumnStatus; + ColumnReader firstColumnStatus; if (columnStatuses.size() > 0) { firstColumnStatus = columnStatuses.iterator().next(); } @@ -404,7 +398,7 @@ public int next() { return 0; } recordsToRead = Math.min(DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH, footer.getBlocks().get(rowGroupIndex).getRowCount() - mockRecordsRead); - for (ValueVector vv : nullFilledVectors ) { + for (final ValueVector vv : nullFilledVectors ) { vv.getMutator().setValueCount( (int) recordsToRead); } mockRecordsRead += recordsToRead; @@ -429,7 +423,7 @@ public int next() { // if we have requested columns that were not found in the file fill their vectors with null // (by simply setting the value counts inside of them, as they start null filled) if (nullFilledVectors != null) { - for (ValueVector vv : nullFilledVectors ) { + for (final ValueVector vv : nullFilledVectors ) { vv.getMutator().setValueCount(firstColumnStatus.getRecordsReadInCurrentPass()); } } @@ -451,13 +445,13 @@ public int next() { } @Override - public void cleanup() { + public void close() { logger.debug("Read {} records out of row group({}) in file '{}'", totalRecordsRead, rowGroupIndex, hadoopPath.toUri().getPath()); // enable this for debugging when it is know that a whole file will be read // limit kills upstream operators once it has enough records, so this assert will fail // assert totalRecordsRead == footer.getBlocks().get(rowGroupIndex).getRowCount(); if (columnStatuses != null) { - for (ColumnReader column : columnStatuses) { + for (final ColumnReader column : columnStatuses) { column.clear(); } columnStatuses.clear(); @@ -467,7 +461,7 @@ public void cleanup() { codecFactory.close(); if (varLengthReader != null) { - for (VarLengthColumn r : varLengthReader.columns) { + for (final VarLengthColumn r : varLengthReader.columns) { r.clear(); } varLengthReader.columns.clear(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java index 4e7d6288f19..5f209703e37 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java @@ -67,7 +67,6 @@ import com.google.common.collect.Sets; public class DrillParquetReader extends AbstractRecordReader { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillParquetReader.class); // same as the DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH in ParquetRecordReader @@ -154,7 +153,7 @@ public static MessageType getProjection(MessageType schema, } // loop through projection columns and add any columns that are missing from parquet schema to columnsNotFound list - outer: for (SchemaPath columnPath : modifiedColumns) { + for (SchemaPath columnPath : modifiedColumns) { boolean notFound = true; for (SchemaPath schemaPath : schemaPaths) { if (schemaPath.contains(columnPath)) { @@ -190,7 +189,7 @@ public static MessageType getProjection(MessageType schema, @Override public void allocate(Map vectorMap) throws OutOfMemoryException { try { - for (ValueVector v : vectorMap.values()) { + for (final ValueVector v : vectorMap.values()) { AllocationHelper.allocate(v, Character.MAX_VALUE, 50, 10); } } catch (NullPointerException e) { @@ -215,7 +214,7 @@ public void setup(OperatorContext context, OutputMutator output) throws Executio projection = schema; } if(columnsNotFound!=null && columnsNotFound.size()>0) { - nullFilledVectors = new ArrayList(); + nullFilledVectors = new ArrayList<>(); for(SchemaPath col: columnsNotFound){ nullFilledVectors.add( (NullableIntVector)output.addField(MaterializedField.create(col, @@ -233,7 +232,7 @@ public void setup(OperatorContext context, OutputMutator output) throws Executio ColumnIOFactory factory = new ColumnIOFactory(false); MessageColumnIO columnIO = factory.getColumnIO(projection, schema); - Map paths = new HashMap(); + Map paths = new HashMap<>(); for (ColumnChunkMetaData md : footer.getBlocks().get(entry.getRowGroupIndex()).getColumns()) { paths.put(md.getPath(), md); @@ -270,7 +269,7 @@ public void setup(OperatorContext context, OutputMutator output) throws Executio } protected void handleAndRaise(String s, Exception e) { - cleanup(); + close(); String message = "Error in drill parquet reader (complex).\nMessage: " + s + "\nParquet Metadata: " + footer; throw new DrillRuntimeException(message, e); @@ -316,7 +315,7 @@ public int next() { // if we have requested columns that were not found in the file fill their vectors with null // (by simply setting the value counts inside of them, as they start null filled) if (nullFilledVectors != null) { - for (ValueVector vv : nullFilledVectors ) { + for (final ValueVector vv : nullFilledVectors) { vv.getMutator().setValueCount(count); } } @@ -325,7 +324,7 @@ public int next() { private int getPercentFilled() { int filled = 0; - for (ValueVector v : primitiveVectors) { + for (final ValueVector v : primitiveVectors) { filled = Math.max(filled, v.getAccessor().getValueCount() * 100 / v.getValueCapacity()); if (v instanceof VariableWidthVector) { filled = Math.max(filled, ((VariableWidthVector) v).getCurrentSizeInBytes() * 100 / ((VariableWidthVector) v).getByteCapacity()); @@ -340,7 +339,7 @@ private int getPercentFilled() { } @Override - public void cleanup() { + public void close() { try { if (pageReadStore != null) { pageReadStore.close(); @@ -351,20 +350,13 @@ public void cleanup() { } } - public void setOperatorContext(OperatorContext operatorContext) { - this.operatorContext = operatorContext; - } + static public class ProjectedColumnType { + public final String projectedColumnName; + public final MessageType type; - static public class ProjectedColumnType{ - ProjectedColumnType(String projectedColumnName, MessageType type){ - this.projectedColumnName=projectedColumnName; - this.type=type; + ProjectedColumnType(String projectedColumnName, MessageType type) { + this.projectedColumnName = projectedColumnName; + this.type = type; } - - public String projectedColumnName; - public MessageType type; - - } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java index 543121fbe32..32fa31d2426 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java @@ -123,7 +123,7 @@ public void setup(OperatorContext context, OutputMutator output) throws Executio @Override public void allocate(Map vectorMap) throws OutOfMemoryException { - for (ValueVector v : vectorMap.values()) { + for (final ValueVector v : vectorMap.values()) { AllocationHelper.allocate(v, Character.MAX_VALUE, 50, 10); } } @@ -146,7 +146,6 @@ public int next() { injector.injectPause(operatorContext.getExecutionControls(), "read-next", logger); try { int i =0; - outside: while (doCurrent || iterator.hasNext()) { if (doCurrent) { doCurrent = false; @@ -175,7 +174,6 @@ public int next() { } @Override - public void cleanup() { + public void close() { } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java index e02b41385c2..82cf04ace9f 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java @@ -17,14 +17,14 @@ */ package org.apache.drill.exec.store.sys; - import java.lang.management.BufferPoolMXBean; + import java.lang.management.ManagementFactory; import java.lang.management.MemoryUsage; import java.util.Iterator; import java.util.List; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; @@ -63,7 +63,7 @@ public Object next() { memoryInfo.direct_current = context.getDrillbitContext().getAllocator().getAllocatedMemory(); - memoryInfo.direct_max = TopLevelAllocator.MAXIMUM_DIRECT_MEMORY; + memoryInfo.direct_max = RootAllocator.getMaxDirect(); return memoryInfo; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/text/DrillTextRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/text/DrillTextRecordReader.java index c59ade9baf3..bc675af1fbc 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/text/DrillTextRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/text/DrillTextRecordReader.java @@ -52,17 +52,14 @@ import com.google.common.collect.Lists; public class DrillTextRecordReader extends AbstractRecordReader { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillTextRecordReader.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillTextRecordReader.class); - static final String COL_NAME = "columns"; + private static final String COL_NAME = "columns"; private org.apache.hadoop.mapred.RecordReader reader; - private List vectors = Lists.newArrayList(); + private final List vectors = Lists.newArrayList(); private byte delimiter; - private int targetRecordCount; private FieldReference ref = new FieldReference(COL_NAME); - private FragmentContext fragmentContext; - private OperatorContext operatorContext; private RepeatedVarCharVector vector; private List columnIds = Lists.newArrayList(); private LongWritable key; @@ -71,9 +68,8 @@ public class DrillTextRecordReader extends AbstractRecordReader { private FileSplit split; private long totalRecordsRead; - public DrillTextRecordReader(FileSplit split, Configuration fsConf, FragmentContext context, char delimiter, - List columns) { - this.fragmentContext = context; + public DrillTextRecordReader(FileSplit split, Configuration fsConf, FragmentContext context, + char delimiter, List columns) { this.delimiter = (byte) delimiter; this.split = split; setColumns(columns); @@ -95,7 +91,6 @@ public DrillTextRecordReader(FileSplit split, Configuration fsConf, FragmentCont Collections.sort(columnIds); numCols = columnIds.size(); } - targetRecordCount = context.getConfig().getInt(ExecConstants.TEXT_LINE_READER_BATCH_SIZE); TextInputFormat inputFormat = new TextInputFormat(); JobConf job = new JobConf(fsConf); @@ -122,14 +117,6 @@ public boolean apply(@Nullable SchemaPath path) { }).isPresent(); } - public OperatorContext getOperatorContext() { - return operatorContext; - } - - public void setOperatorContext(OperatorContext operatorContext) { - this.operatorContext = operatorContext; - } - @Override public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException { MaterializedField field = MaterializedField.create(ref, Types.repeated(TypeProtos.MinorType.VARCHAR)); @@ -155,6 +142,7 @@ public int next() { int batchSize = 0; try { int recordCount = 0; + final RepeatedVarCharVector.Mutator mutator = vector.getMutator(); while (recordCount < Character.MAX_VALUE && batchSize < 200*1000 && reader.next(key, value)) { int start; int end = -1; @@ -162,7 +150,7 @@ public int next() { // index of the scanned field int p = 0; int i = 0; - vector.getMutator().startNewValue(recordCount); + mutator.startNewValue(recordCount); // Process each field in this line while (end < value.getLength() - 1) { if(numCols > 0 && p >= numCols) { @@ -178,24 +166,24 @@ public int next() { } } if (numCols > 0 && i++ < columnIds.get(p)) { - vector.getMutator().addSafe(recordCount, value.getBytes(), start + 1, 0); + mutator.addSafe(recordCount, value.getBytes(), start + 1, 0); continue; } p++; - vector.getMutator().addSafe(recordCount, value.getBytes(), start + 1, end - start - 1); + mutator.addSafe(recordCount, value.getBytes(), start + 1, end - start - 1); batchSize += end - start; } recordCount++; totalRecordsRead++; } - for (ValueVector v : vectors) { + for (final ValueVector v : vectors) { v.getMutator().setValueCount(recordCount); } - vector.getMutator().setValueCount(recordCount); -// logger.debug("text scan batch size {}", batchSize); + mutator.setValueCount(recordCount); + // logger.debug("text scan batch size {}", batchSize); return recordCount; } catch(Exception e) { - cleanup(); + close(); handleAndRaise("Failure while parsing text. Parser was at record: " + (totalRecordsRead + 1), e); } @@ -229,7 +217,7 @@ public int find(Text text, byte delimiter, int start) { } @Override - public void cleanup() { + public void close() { try { if (reader != null) { reader.close(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java index d8979d244e5..fdb318b9156 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java @@ -50,10 +50,11 @@ public Class getSiteClass() { @Override public void injectUnchecked(final ExecutionControls executionControls, final String desc) { - Preconditions.checkNotNull(executionControls); - final ExceptionInjection exceptionInjection = executionControls.lookupExceptionInjection(this, desc); - if (exceptionInjection != null) { - exceptionInjection.throwUnchecked(); + if (executionControls != null) { + final ExceptionInjection exceptionInjection = executionControls.lookupExceptionInjection(this, desc); + if (exceptionInjection != null) { + exceptionInjection.throwUnchecked(); + } } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java index e5180420b6b..622e2d2df62 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java @@ -21,22 +21,22 @@ import org.apache.drill.exec.vector.complex.RepeatedVariableWidthVectorLike; public class AllocationHelper { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AllocationHelper.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AllocationHelper.class); - public static void allocate(ValueVector v, int valueCount, int bytesPerValue){ + public static void allocate(ValueVector v, int valueCount, int bytesPerValue) { allocate(v, valueCount, bytesPerValue, 5); } public static void allocatePrecomputedChildCount(ValueVector v, int valueCount, int bytesPerValue, int childValCount){ - if(v instanceof FixedWidthVector){ + if(v instanceof FixedWidthVector) { ((FixedWidthVector) v).allocateNew(valueCount); } else if (v instanceof VariableWidthVector) { ((VariableWidthVector) v).allocateNew(valueCount * bytesPerValue, valueCount); - }else if(v instanceof RepeatedFixedWidthVectorLike){ + } else if(v instanceof RepeatedFixedWidthVectorLike) { ((RepeatedFixedWidthVectorLike) v).allocateNew(valueCount, childValCount); - }else if(v instanceof RepeatedVariableWidthVectorLike){ + } else if(v instanceof RepeatedVariableWidthVectorLike) { ((RepeatedVariableWidthVectorLike) v).allocateNew(childValCount * bytesPerValue, valueCount, childValCount); - }else{ + } else { v.allocateNew(); } } @@ -58,5 +58,4 @@ public static void allocateNew(ValueVector v, int valueCount) { v.allocateNew(); } } - } 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 579eed62f35..42e0972423d 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 @@ -24,20 +24,34 @@ public abstract class BaseDataValueVector extends BaseValueVector { + protected final static byte[] emptyByteArray = new byte[]{}; // Nullable vectors use this + protected DrillBuf data; public BaseDataValueVector(MaterializedField field, BufferAllocator allocator) { super(field, allocator); - this.data = allocator.getEmpty(); + data = allocator.getEmpty(); } @Override public void clear() { - data.release(); + if (data != null) { + data.release(); + } data = allocator.getEmpty(); super.clear(); } + @Override + public void close() { + clear(); + if (data != null) { + data.release(); + data = null; + } + super.close(); + } + @Override public DrillBuf[] getBuffers(boolean clear) { DrillBuf[] out; @@ -47,7 +61,7 @@ public DrillBuf[] getBuffers(boolean clear) { out = new DrillBuf[]{data}; if (clear) { data.readerIndex(0); - data.retain(); + data.retain(1); } } if (clear) { @@ -56,6 +70,7 @@ public DrillBuf[] getBuffers(boolean clear) { return out; } + @Override public int getBufferSize() { if (getAccessor().getValueCount() == 0) { return 0; 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 cc287c48a9a..7b3ab414f12 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 @@ -17,10 +17,13 @@ */ package org.apache.drill.exec.vector; +import io.netty.buffer.DrillBuf; + import java.util.Iterator; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; + import org.apache.drill.common.expression.FieldReference; import org.apache.drill.exec.memory.BufferAllocator; import org.apache.drill.exec.proto.UserBitShared.SerializedField; @@ -91,10 +94,10 @@ public abstract static class BaseMutator implements ValueVector.Mutator { protected BaseMutator() { } @Override - public void generateTestData(int values) { } + public void generateTestData(int values) {} //TODO: consider making mutator stateless(if possible) on another issue. - public void reset() { } + public void reset() {} } @Override @@ -102,5 +105,14 @@ public Iterator iterator() { return Iterators.emptyIterator(); } + public static boolean checkBufRefs(final ValueVector vv) { + for(final DrillBuf buffer : vv.getBuffers(false)) { + if (buffer.refCnt() <= 0) { + throw new IllegalStateException("zero refcount"); + } + } + + return true; + } } 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 054ef826123..726e7eeb9bd 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 @@ -79,12 +79,14 @@ public void setInitialCapacity(final int valueCount) { allocationSizeInBytes = getSizeFromCount(valueCount); } + @Override public void allocateNew() { if (!allocateNewSafe()) { throw new OutOfMemoryRuntimeException(); } } + @Override public boolean allocateNewSafe() { long curAllocationSize = allocationSizeInBytes; if (allocationMonitor > 10) { @@ -109,6 +111,7 @@ public boolean allocateNewSafe() { * @param valueCount * The number of values which can be contained within this vector. */ + @Override public void allocateNew(int valueCount) { final int size = getSizeFromCount(valueCount); allocateBytes(size); @@ -119,7 +122,7 @@ private void allocateBytes(final long size) { throw new OversizedAllocationException("Requested amount of memory is more than max allowed allocation size"); } - final int curSize = (int)size; + final int curSize = (int) size; clear(); data = allocator.buffer(curSize); zeroVector(); @@ -157,7 +160,7 @@ public int load(int valueCount, DrillBuf buf) { clear(); this.valueCount = valueCount; int len = getSizeFromCount(valueCount); - data = buf.slice(0, len); + data = (DrillBuf) buf.slice(0, len); data.retain(); return len; } @@ -182,21 +185,27 @@ public void load(SerializedField metadata, DrillBuf buffer) { assert metadata.getBufferLength() == loaded; } + @Override public Mutator getMutator() { return new Mutator(); } + @Override public Accessor getAccessor() { return new Accessor(); } + @Override public TransferPair getTransferPair() { return new TransferImpl(getField()); } + + @Override public TransferPair getTransferPair(FieldReference ref) { return new TransferImpl(getField().withPath(ref)); } + @Override public TransferPair makeTransferPair(ValueVector to) { return new TransferImpl((BitVector) to); } @@ -204,8 +213,11 @@ public TransferPair makeTransferPair(ValueVector to) { public void transferTo(BitVector target) { target.clear(); + if (target.data != null) { + target.data.release(); + } target.data = data; - target.data.retain(); + target.data.retain(1); target.valueCount = valueCount; clear(); } @@ -218,8 +230,11 @@ public void splitAndTransferTo(int startIndex, int length, BitVector target) { if (offset == 0) { target.clear(); // slice - target.data = (DrillBuf) this.data.slice(firstByte, byteSize); - target.data.retain(); + if (target.data != null) { + target.data.release(); + } + target.data = (DrillBuf) data.slice(firstByte, byteSize); + target.data.retain(1); } else { // Copy data // When the first bit starts from the middle of a byte (offset != 0), copy data from src BitVector. @@ -230,7 +245,7 @@ public void splitAndTransferTo(int startIndex, int length, BitVector target) { target.clear(); target.allocateNew(length); // TODO maybe do this one word at a time, rather than byte? - for (int i = 0; i < byteSize - 1; i++) { + for(int i = 0; i < byteSize - 1; i++) { target.data.setByte(i, (((this.data.getByte(firstByte + i) & 0xFF) >>> offset) + (this.data.getByte(firstByte + i + 1) << (8 - offset)))); } if (length % 8 != 0) { @@ -254,14 +269,17 @@ public TransferImpl(BitVector to) { this.to = to; } + @Override public BitVector getTo() { return to; } + @Override public void transfer() { transferTo(to); } + @Override public void splitAndTransfer(int startIndex, int length) { splitAndTransferTo(startIndex, length, to); } @@ -386,6 +404,7 @@ public void setSafe(int index, NullableBitHolder holder) { set(index, holder.value); } + @Override public final void setValueCount(int valueCount) { int currentValueCapacity = getValueCapacity(); BitVector.this.valueCount = valueCount; @@ -404,7 +423,7 @@ public final void setValueCount(int valueCount) { @Override public final void generateTestData(int values) { boolean even = true; - for (int i = 0; i < values; i++, even = !even) { + for(int i = 0; i < values; i++, even = !even) { if (even) { set(i, 1); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java index 3948163a545..9d4b08d0d64 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java @@ -17,10 +17,10 @@ */ package org.apache.drill.exec.vector; -import io.netty.buffer.DrillBuf; - import java.io.Closeable; +import io.netty.buffer.DrillBuf; + import org.apache.drill.common.expression.FieldReference; import org.apache.drill.exec.memory.OutOfMemoryRuntimeException; import org.apache.drill.exec.proto.UserBitShared.SerializedField; @@ -64,8 +64,9 @@ public interface ValueVector extends Closeable, Iterable { int getValueCapacity(); /** - * Alternative to clear(). Allows use as closeable in try-with-resources. + * Alternative to clear(). Allows use as an AutoCloseable in try-with-resources. */ + @Override void close(); /** @@ -126,9 +127,8 @@ public interface ValueVector extends Closeable, Iterable { * Return the underlying buffers associated with this vector. Note that this doesn't impact the reference counts for * this buffer so it only should be used for in-context access. Also note that this buffer changes regularly thus * external classes shouldn't hold a reference to it (unless they change it). - * - * @param clear - * Whether to clear vector + * @param clear Whether to clear vector before returning; the buffers will still be refcounted; + * but the returned array will be the only reference to them * * @return The underlying {@link io.netty.buffer.DrillBuf buffers} that is used by this vector instance. */ @@ -189,5 +189,4 @@ interface Mutator { @Deprecated void generateTestData(int values); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/AbstractContainerVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/AbstractContainerVector.java index d14dca6bebf..dd6b8029086 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/AbstractContainerVector.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/AbstractContainerVector.java @@ -45,7 +45,7 @@ * This class implements common functionality of composite vectors. */ public abstract class AbstractContainerVector implements ValueVector { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractContainerVector.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractContainerVector.class); protected final MaterializedField field; protected final BufferAllocator allocator; @@ -97,7 +97,8 @@ public String apply(MaterializedField field) { */ @Override public void close() { - for (ValueVector vector:(Iterable)this) { + final Iterable vectors = (Iterable) this; + for (final ValueVector vector : vectors) { vector.close(); } } 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 1df4b81e64e..ced04b4d9bf 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 @@ -36,6 +36,7 @@ * Base class for MapVectors. Currently used by RepeatedMapVector and MapVector */ public abstract class AbstractMapVector extends AbstractContainerVector { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractContainerVector.class); // Maintains a map with key as field name and value is the vector itself private final MapWithOrdinal vectors = new MapWithOrdinal<>(); @@ -45,13 +46,23 @@ protected AbstractMapVector(MaterializedField field, BufferAllocator allocator, // create the hierarchy of the child vectors based on the materialized field for (MaterializedField child : field.getChildren()) { if (!child.equals(BaseRepeatedValueVector.OFFSETS_FIELD)) { - String fieldName = child.getLastName(); - ValueVector v = TypeHelper.getNewVector(child, allocator, callBack); + final String fieldName = child.getLastName(); + final ValueVector v = TypeHelper.getNewVector(child, allocator, callBack); putVector(fieldName, v); } } } + @Override + public void close() { + for(final ValueVector valueVector : vectors.values()) { + valueVector.close(); + } + vectors.clear(); + + super.close(); + } + @Override public boolean allocateNewSafe() { /* boolean to keep track if all the memory allocation were successful @@ -61,8 +72,7 @@ public boolean allocateNewSafe() { */ boolean success = false; try { - - for (ValueVector v : vectors.values()) { + for (final ValueVector v : vectors.values()) { if (!v.allocateNewSafe()) { return false; } @@ -104,13 +114,14 @@ public boolean allocateNewSafe() { * * @return resultant {@link org.apache.drill.exec.vector.ValueVector} */ + @Override public T addOrGet(String name, TypeProtos.MajorType type, Class clazz) { final ValueVector existing = getChild(name); boolean create = false; if (existing == null) { create = true; } else if (clazz.isAssignableFrom(existing.getClass())) { - return (T)existing; + return (T) existing; } else if (nullFilled(existing)) { existing.clear(); create = true; @@ -128,7 +139,7 @@ public T addOrGet(String name, TypeProtos.MajorType type } private boolean nullFilled(ValueVector vector) { - for (int r=0; r corresponding to the given * field name if exists or null. */ + @Override public T getChild(String name, Class clazz) { - ValueVector v = vectors.get(name.toLowerCase()); + final ValueVector v = vectors.get(name.toLowerCase()); if (v == null) { return null; } @@ -171,7 +183,7 @@ protected void putChild(String name, ValueVector vector) { * @param vector vector to be inserted */ protected void putVector(String name, ValueVector vector) { - ValueVector old = vectors.put( + final ValueVector old = vectors.put( Preconditions.checkNotNull(name, "field name cannot be null").toLowerCase(), Preconditions.checkNotNull(vector, "vector cannot be null") ); @@ -191,6 +203,7 @@ protected Collection getChildren() { /** * Returns the number of underlying child vectors. */ + @Override public int size() { return vectors.size(); } @@ -204,8 +217,8 @@ public Iterator iterator() { * Returns a list of scalar child vectors recursing the entire vector hierarchy. */ public List getPrimitiveVectors() { - List primitiveVectors = Lists.newArrayList(); - for (ValueVector v : vectors.values()) { + final List primitiveVectors = Lists.newArrayList(); + for (final ValueVector v : vectors.values()) { if (v instanceof AbstractMapVector) { AbstractMapVector mapVector = (AbstractMapVector) v; primitiveVectors.addAll(mapVector.getPrimitiveVectors()); @@ -219,6 +232,7 @@ public List getPrimitiveVectors() { /** * Returns a vector with its corresponding ordinal mapping if field exists or null. */ + @Override public VectorWithOrdinal getChildVectorWithOrdinal(String name) { final int ordinal = vectors.getOrdinal(name.toLowerCase()); if (ordinal < 0) { @@ -230,13 +244,13 @@ public VectorWithOrdinal getChildVectorWithOrdinal(String name) { @Override public DrillBuf[] getBuffers(boolean clear) { - List buffers = Lists.newArrayList(); + final List buffers = Lists.newArrayList(); - for (ValueVector vector : vectors.values()) { - for (DrillBuf buf : vector.getBuffers(false)) { + for (final ValueVector vector : vectors.values()) { + for (final DrillBuf buf : vector.getBuffers(false)) { buffers.add(buf); if (clear) { - buf.retain(); + buf.retain(1); } } if (clear) { @@ -251,20 +265,11 @@ public DrillBuf[] getBuffers(boolean clear) { public int getBufferSize() { int actualBufSize = 0 ; - for (ValueVector v : vectors.values()) { - for (DrillBuf buf : v.getBuffers(false)) { + for (final ValueVector v : vectors.values()) { + for (final DrillBuf buf : v.getBuffers(false)) { actualBufSize += buf.writerIndex(); } } return actualBufSize; } - - @Override - public void close() { - for(final ValueVector valueVector : vectors.values()) { - valueVector.close(); - } - - super.close(); - } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java index 3032aacadbb..8c897d8eeb6 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java @@ -22,6 +22,7 @@ import io.netty.buffer.DrillBuf; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -50,7 +51,7 @@ import com.google.common.base.Preconditions; public class MapVector extends AbstractMapVector { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapVector.class); + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapVector.class); public final static MajorType TYPE = MajorType.newBuilder().setMinorType(MinorType.MAP).setMode(DataMode.REQUIRED).build(); @@ -102,7 +103,7 @@ public Iterator fieldNameIterator() { @Override public void setInitialCapacity(int numRecords) { final Iterable container = this; - for (ValueVector v : container) { + for (final ValueVector v : container) { v.setInitialCapacity(numRecords); } } @@ -113,7 +114,7 @@ public int getBufferSize() { return 0; } long buffer = 0; - for (ValueVector v : (Iterable)this) { + for (final ValueVector v : (Iterable)this) { buffer += v.getBufferSize(); } @@ -136,7 +137,7 @@ public TransferPair getTransferPair() { @Override public TransferPair makeTransferPair(ValueVector to) { - return new MapTransferPair(this, (MapVector)to); + return new MapTransferPair(this, (MapVector) to); } @Override @@ -179,7 +180,7 @@ protected MapTransferPair(MapVector from, MapVector to, boolean allocate) { // (This is similar to what happens in ScanBatch where the children cannot be added till they are // read). To take care of this, we ensure that the hashCode of the MaterializedField does not // include the hashCode of the children but is based only on MaterializedField$key. - ValueVector newVector = to.addOrGet(child, vector.getField().getType(), vector.getClass()); + final ValueVector newVector = to.addOrGet(child, vector.getField().getType(), vector.getClass()); if (allocate && to.size() != preSize) { newVector.allocateNew(); } @@ -187,10 +188,9 @@ protected MapTransferPair(MapVector from, MapVector to, boolean allocate) { } } - @Override public void transfer() { - for (TransferPair p : pairs) { + for (final TransferPair p : pairs) { p.transfer(); } to.valueCount = from.valueCount; @@ -216,7 +216,6 @@ public void splitAndTransfer(int startIndex, int length) { } to.getMutator().setValueCount(length); } - } @Override @@ -311,7 +310,6 @@ public void get(int index, ComplexHolder holder) { public int getValueCount() { return valueCount; } - } public ValueVector getVectorById(int id) { @@ -322,7 +320,7 @@ public class Mutator extends BaseValueVector.BaseMutator { @Override public void setValueCount(int valueCount) { - for (ValueVector v : getChildren()) { + for (final ValueVector v : getChildren()) { v.getMutator().setValueCount(valueCount); } MapVector.this.valueCount = valueCount; @@ -337,17 +335,19 @@ public void generateTestData(int values) { } @Override public void clear() { - valueCount = 0; - for (ValueVector v : getChildren()) { + for (final ValueVector v : getChildren()) { v.clear(); } + valueCount = 0; } @Override public void close() { - for (final ValueVector v : getChildren()) { + final Collection vectors = getChildren(); + for (final ValueVector v : vectors) { v.close(); } + vectors.clear(); valueCount = 0; super.close(); 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 85e4d1d1837..bd6ed93a19c 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 @@ -47,7 +47,6 @@ import org.apache.drill.exec.vector.complex.impl.RepeatedListReaderImpl; import org.apache.drill.exec.vector.complex.reader.FieldReader; - public class RepeatedListVector extends AbstractContainerVector implements RepeatedValueVector, RepeatedFixedWidthVectorLike { @@ -180,7 +179,7 @@ public DelegateRepeatedVector(SchemaPath path, BufferAllocator allocator) { public DelegateRepeatedVector(MaterializedField field, BufferAllocator allocator) { super(field, allocator); - this.emptyPopulator = new EmptyValuePopulator(getOffsetVector()); + emptyPopulator = new EmptyValuePopulator(getOffsetVector()); } @Override @@ -423,5 +422,4 @@ public VectorWithOrdinal getChildVectorWithOrdinal(String name) { public void copyFromSafe(int fromIndex, int thisIndex, RepeatedListVector from) { delegate.copyFromSafe(fromIndex, thisIndex, from.delegate); } - } 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 97f5b396632..503cc319f51 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 @@ -54,13 +54,13 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Maps; -public class RepeatedMapVector extends AbstractMapVector implements RepeatedValueVector, RepeatedFixedWidthVectorLike { - - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RepeatedMapVector.class); +public class RepeatedMapVector extends AbstractMapVector + implements RepeatedValueVector, RepeatedFixedWidthVectorLike { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RepeatedMapVector.class); public final static MajorType TYPE = MajorType.newBuilder().setMinorType(MinorType.MAP).setMode(DataMode.REPEATED).build(); - final UInt4Vector offsets; // offsets to start of each record (considering record indices are 0-indexed) + private final UInt4Vector offsets; // offsets to start of each record (considering record indices are 0-indexed) private final RepeatedMapReaderImpl reader = new RepeatedMapReaderImpl(RepeatedMapVector.this); private final RepeatedMapAccessor accessor = new RepeatedMapAccessor(); private final Mutator mutator = new Mutator(); @@ -123,7 +123,7 @@ public Iterator fieldNameIterator() { @Override public List getPrimitiveVectors() { - List primitiveVectors = super.getPrimitiveVectors(); + final List primitiveVectors = super.getPrimitiveVectors(); primitiveVectors.add(offsets); return primitiveVectors; } @@ -134,7 +134,7 @@ public int getBufferSize() { return 0; } long buffer = offsets.getBufferSize(); - for (ValueVector v : (Iterable)this) { + for (final ValueVector v : (Iterable) this) { buffer += v.getBufferSize(); } return (int) buffer; @@ -142,8 +142,8 @@ public int getBufferSize() { @Override public void close() { - super.close(); offsets.close(); + super.close(); } @Override @@ -170,13 +170,13 @@ public MapSingleCopier(RepeatedMapVector from, MapVector to) { int i = 0; ValueVector vector; - for (String child:from.getChildFieldNames()) { + for (final String child:from.getChildFieldNames()) { int preSize = to.size(); vector = from.getChild(child); if (vector == null) { continue; } - ValueVector newVector = to.addOrGet(child, vector.getField().getType(), vector.getClass()); + final ValueVector newVector = to.addOrGet(child, vector.getField().getType(), vector.getClass()); if (to.size() != preSize) { newVector.allocateNew(); } @@ -262,13 +262,13 @@ public SingleMapTransferPair(RepeatedMapVector from, MapVector to, boolean alloc this.pairs = new TransferPair[from.size()]; int i = 0; ValueVector vector; - for (String child:from.getChildFieldNames()) { + for (final String child : from.getChildFieldNames()) { int preSize = to.size(); vector = from.getChild(child); if (vector == null) { continue; } - ValueVector newVector = to.addOrGet(child, vector.getField().getType(), vector.getClass()); + final ValueVector newVector = to.addOrGet(child, vector.getField().getType(), vector.getClass()); if (allocate && to.size() != preSize) { newVector.allocateNew(); } @@ -305,7 +305,6 @@ public void splitAndTransfer(int startIndex, int length) { } to.getMutator().setValueCount(length); } - } private static class RepeatedMapTransferPair implements TransferPair{ @@ -328,22 +327,22 @@ public RepeatedMapTransferPair(RepeatedMapVector from, RepeatedMapVector to, boo this.pairs = new TransferPair[from.size()]; int i = 0; - ValueVector vector; - for (String child:from.getChildFieldNames()) { - int preSize = to.size(); - vector = from.getChild(child); + for (final String child : from.getChildFieldNames()) { + final int preSize = to.size(); + final ValueVector vector = from.getChild(child); if (vector == null) { continue; } - ValueVector newVector = to.addOrGet(child, vector.getField().getType(), vector.getClass()); + + final ValueVector newVector = to.addOrGet(child, vector.getField().getType(), vector.getClass()); if (to.size() != preSize) { newVector.allocateNew(); } + pairs[i++] = vector.makeTransferPair(newVector); } } - @Override public void transfer() { from.offsets.transferTo(to.offsets); @@ -370,7 +369,7 @@ public void copyValueSafe(int srcIndex, int destIndex) { p.copyValueSafe(i, newIndex); } } - to.offsets.getMutator().setSafe(destIndex+1, newIndex); + to.offsets.getMutator().setSafe(destIndex + 1, newIndex); } @Override @@ -379,27 +378,25 @@ public void splitAndTransfer(final int groupStart, final int groups) { final UInt4Vector.Mutator m = to.offsets.getMutator(); final int startPos = a.get(groupStart); - final int endPos = a.get(groupStart+groups); + final int endPos = a.get(groupStart + groups); final int valuesToCopy = endPos - startPos; to.offsets.clear(); to.offsets.allocateNew(groups + 1); int normalizedPos; - for (int i=0; i < groups+1; i++) { - normalizedPos = a.get(groupStart+i) - startPos; + for (int i = 0; i < groups + 1; i++) { + normalizedPos = a.get(groupStart + i) - startPos; m.set(i, normalizedPos); } m.setValueCount(groups + 1); to.emptyPopulator.populate(groups); - for (TransferPair p : pairs) { + for (final TransferPair p : pairs) { p.splitAndTransfer(startPos, valuesToCopy); } - } - } @@ -414,7 +411,7 @@ public void copyFromSafe(int fromIndex, int thisIndex, RepeatedMapVector from) { @Override public int getValueCapacity() { - return offsets.getValueCapacity()-1; + return offsets.getValueCapacity() - 1; } @Override @@ -424,8 +421,8 @@ public RepeatedMapAccessor getAccessor() { @Override public DrillBuf[] getBuffers(boolean clear) { - int expectedBufferSize = getBufferSize(); - int actualBufferSize = super.getBufferSize(); + final int expectedBufferSize = getBufferSize(); + final int actualBufferSize = super.getBufferSize(); Preconditions.checkArgument(expectedBufferSize == actualBufferSize + offsets.getBufferSize()); return ArrayUtils.addAll(offsets.getBuffers(clear), super.getBuffers(clear)); @@ -440,7 +437,7 @@ public void load(SerializedField metadata, DrillBuf buffer) { offsets.load(offsetField, buffer); int bufOffset = offsetField.getBufferLength(); - for (int i=1; i list = new JsonStringArrayList(); - int end = offsets.getAccessor().get(index+1); + final int end = offsets.getAccessor().get(index+1); String fieldName; for (int i = offsets.getAccessor().get(index); i < end; i++) { - Map vv = Maps.newLinkedHashMap(); - for (MaterializedField field:getField().getChildren()) { + final Map vv = Maps.newLinkedHashMap(); + for (final MaterializedField field : getField().getChildren()) { if (!field.equals(BaseRepeatedValueVector.OFFSETS_FIELD)) { fieldName = field.getLastName(); final Object value = getChild(fieldName).getAccessor().getObject(i); @@ -530,21 +527,24 @@ public boolean isNull(int index) { } public void get(int index, RepeatedMapHolder holder) { - assert index < getValueCapacity() : String.format("Attempted to access index %d when value capacity is %d", index, getValueCapacity()); - holder.start = offsets.getAccessor().get(index); - holder.end = offsets.getAccessor().get(index+1); + assert index < getValueCapacity() : + String.format("Attempted to access index %d when value capacity is %d", + index, getValueCapacity()); + final UInt4Vector.Accessor offsetsAccessor = offsets.getAccessor(); + holder.start = offsetsAccessor.get(index); + holder.end = offsetsAccessor.get(index + 1); } public void get(int index, ComplexHolder holder) { - FieldReader reader = getReader(); + final FieldReader reader = getReader(); reader.setPosition(index); holder.reader = reader; } public void get(int index, int arrayIndex, ComplexHolder holder) { - RepeatedMapHolder h = new RepeatedMapHolder(); + final RepeatedMapHolder h = new RepeatedMapHolder(); get(index, h); - int offset = h.start + arrayIndex; + final int offset = h.start + arrayIndex; if (offset >= h.end) { holder.reader = NullReader.INSTANCE; @@ -553,36 +553,33 @@ public void get(int index, int arrayIndex, ComplexHolder holder) { holder.reader = reader; } } - } - public class Mutator implements RepeatedMutator { - @Override public void startNewValue(int index) { - emptyPopulator.populate(index+1); - offsets.getMutator().setSafe(index+1, offsets.getAccessor().get(index)); + emptyPopulator.populate(index + 1); + offsets.getMutator().setSafe(index + 1, offsets.getAccessor().get(index)); } @Override public void setValueCount(int topLevelValueCount) { emptyPopulator.populate(topLevelValueCount); - offsets.getMutator().setValueCount(topLevelValueCount == 0 ? 0 : topLevelValueCount+1); + offsets.getMutator().setValueCount(topLevelValueCount == 0 ? 0 : topLevelValueCount + 1); int childValueCount = offsets.getAccessor().get(topLevelValueCount); - for (ValueVector v : getChildren()) { + for (final ValueVector v : getChildren()) { v.getMutator().setValueCount(childValueCount); } } @Override - public void reset() { } + public void reset() {} @Override - public void generateTestData(int values) { } + public void generateTestData(int values) {} public int add(int index) { - final int prevEnd = offsets.getAccessor().get(index+1); + final int prevEnd = offsets.getAccessor().get(index + 1); offsets.getMutator().setSafe(index + 1, prevEnd + 1); return prevEnd; } @@ -593,7 +590,7 @@ public void clear() { getMutator().reset(); offsets.clear(); - for(ValueVector vector:getChildren()) { + for(final ValueVector vector : getChildren()) { vector.clear(); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/DrillBufInputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/DrillBufInputStream.java index 1061a5cdf2c..4bf61d89c08 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/DrillBufInputStream.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/DrillBufInputStream.java @@ -28,7 +28,7 @@ * An InputStream that wraps a DrillBuf and implements the seekable interface. */ public class DrillBufInputStream extends ByteBufInputStream implements Seekable { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillBufInputStream.class); + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillBufInputStream.class); private final DrillBuf buffer; @@ -52,7 +52,8 @@ public boolean seekToNewSource(long targetPos) throws IOException { return false; } - public static DrillBufInputStream getStream(int start, int end, DrillBuf buffer){ + // Does not adopt the buffer + public static DrillBufInputStream getStream(int start, int end, DrillBuf buffer) { DrillBuf buf = buffer.slice(start, end - start); return new DrillBufInputStream(buf, end - start); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java index 5c03c028198..603776daba9 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java @@ -27,7 +27,6 @@ import org.apache.drill.common.expression.PathSegment; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.exec.physical.base.GroupScan; -import org.apache.drill.exec.store.easy.json.JsonProcessor; import org.apache.drill.exec.store.easy.json.reader.BaseJsonProcessor; import org.apache.drill.exec.vector.complex.fn.VectorOutput.ListVectorOutput; import org.apache.drill.exec.vector.complex.fn.VectorOutput.MapVectorOutput; @@ -36,16 +35,14 @@ import org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter; import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter; -import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.JsonToken; import com.fasterxml.jackson.databind.JsonNode; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; public class JsonReader extends BaseJsonProcessor { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonReader.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonReader.class); public final static int MAX_RECORD_SIZE = 128 * 1024; private final WorkingBuffer workingBuffer; @@ -466,7 +463,7 @@ private void handleString(JsonParser parser, ListWriter writer) throws IOExcepti writer.varChar().writeVarChar(0, workingBuffer.prepareVarCharHolder(parser.getText()), workingBuffer.getBuf()); } - private void writeData(ListWriter list) throws IOException { + private void writeData(ListWriter list) { list.start(); outside: while (true) { try { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/AbstractBaseWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/AbstractBaseWriter.java index ec8c00be43c..06864203094 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/AbstractBaseWriter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/AbstractBaseWriter.java @@ -20,33 +20,35 @@ import org.apache.drill.exec.vector.complex.writer.FieldWriter; -abstract class AbstractBaseWriter implements FieldWriter{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractBaseWriter.class); +abstract class AbstractBaseWriter implements FieldWriter { + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractBaseWriter.class); final FieldWriter parent; private int index; public AbstractBaseWriter(FieldWriter parent) { - super(); this.parent = parent; } + @Override public FieldWriter getParent() { return parent; } - public boolean isRoot(){ + public boolean isRoot() { return parent == null; } - int idx(){ + int idx() { return index; } - public void setPosition(int index){ + @Override + public void setPosition(int index) { this.index = index; } - public void end(){ + @Override + public void end() { } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java index a4a35e29c0e..88a56f8707e 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java @@ -29,12 +29,12 @@ import com.google.common.base.Preconditions; -public class ComplexWriterImpl extends AbstractFieldWriter implements ComplexWriter{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ComplexWriterImpl.class); +public class ComplexWriterImpl extends AbstractFieldWriter implements ComplexWriter { +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ComplexWriterImpl.class); - SingleMapWriter mapRoot; - SingleListWriter listRoot; - MapVector container; + private SingleMapWriter mapRoot; + private SingleListWriter listRoot; + private final MapVector container; Mode mode = Mode.INIT; private final String name; @@ -47,6 +47,7 @@ public ComplexWriterImpl(String name, MapVector container){ this.container = container; } + @Override public MaterializedField getField() { return container.getField(); } @@ -60,10 +61,21 @@ private void check(Mode... modes){ StateTool.check(mode, modes); } + @Override public void reset(){ setPosition(0); } + @Override + public void close() throws Exception { + clear(); + mapRoot.close(); + if (listRoot != null) { + listRoot.close(); + } + } + + @Override public void clear(){ switch(mode){ case MAP: @@ -75,6 +87,7 @@ public void clear(){ } } + @Override public void setValueCount(int count){ switch(mode){ case MAP: @@ -146,9 +159,9 @@ public MapWriter rootAsMap() { @Override public void allocate() { - if(mapRoot != null){ + if(mapRoot != null) { mapRoot.allocate(); - }else if(listRoot != null){ + } else if(listRoot != null) { listRoot.allocate(); } } @@ -184,7 +197,7 @@ public VectorAccessibleFacade(VectorContainer vc) { @Override public T addOrGet(String name, MajorType type, Class clazz) { - ValueVector v = vc.addOrGet(name, type, clazz); + final ValueVector v = vc.addOrGet(name, type, clazz); putChild(name, v); return this.typeify(v, clazz); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java index 6b6ab46a1b7..92ff4b20886 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java @@ -26,19 +26,20 @@ import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter; public class VectorContainerWriter extends AbstractFieldWriter implements ComplexWriter { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorContainerWriter.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorContainerWriter.class); - SingleMapWriter mapRoot; - SpecialMapVector mapVector; - OutputMutator mutator; + private final SingleMapWriter mapRoot; + private final SpecialMapVector mapVector; + private final OutputMutator mutator; public VectorContainerWriter(OutputMutator mutator) { super(null); this.mutator = mutator; - this.mapVector = new SpecialMapVector(); - this.mapRoot = new SingleMapWriter(mapVector, this); + mapVector = new SpecialMapVector(); + mapRoot = new SingleMapWriter(mapVector, this); } + @Override public MaterializedField getField() { return mapVector.getField(); } @@ -52,10 +53,19 @@ public MapVector getMapVector() { return mapVector; } + @Override public void reset() { setPosition(0); } + @Override + public void close() throws Exception { + clear(); + mapRoot.close(); + mapVector.close(); + } + + @Override public void clear() { mapRoot.clear(); } @@ -64,6 +74,7 @@ public SingleMapWriter getWriter() { return mapRoot; } + @Override public void setValueCount(int count) { mapRoot.setValueCount(count); } @@ -88,15 +99,13 @@ public SpecialMapVector() { @Override public T addOrGet(String name, MajorType type, Class clazz) { try { - ValueVector v = mutator.addField(MaterializedField.create(name, type), clazz); + final ValueVector v = mutator.addField(MaterializedField.create(name, type), clazz); putChild(name, v); return this.typeify(v, clazz); } catch (SchemaChangeException e) { throw new IllegalStateException(e); } - } - } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/writer/FieldWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/writer/FieldWriter.java index 3faa4f700f9..dd8da679bd1 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/writer/FieldWriter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/writer/FieldWriter.java @@ -21,9 +21,7 @@ import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter; import org.apache.drill.exec.vector.complex.writer.BaseWriter.ScalarWriter; - - public interface FieldWriter extends MapWriter, ListWriter, ScalarWriter { - public void allocate(); - public void clear(); + void allocate(); + void clear(); } \ No newline at end of file diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java index d52cb5d5327..f2eff15d82f 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java @@ -71,18 +71,12 @@ public AbstractDataCollector(AtomicInteger parentAccounter, final boolean spooling = collector.getIsSpooling(); - try { - - for (int i = 0; i < numBuffers; i++) { - if (spooling) { - buffers[i] = new SpoolingRawBatchBuffer(context, bufferCapacity, collector.getOppositeMajorFragmentId(), i); - } else { - buffers[i] = new UnlimitedRawBatchBuffer(context, bufferCapacity, collector.getOppositeMajorFragmentId()); - } + for (int i = 0; i < numBuffers; i++) { + if (spooling) { + buffers[i] = new SpoolingRawBatchBuffer(context, bufferCapacity, collector.getOppositeMajorFragmentId(), i); + } else { + buffers[i] = new UnlimitedRawBatchBuffer(context, bufferCapacity, collector.getOppositeMajorFragmentId()); } - } catch (IOException | OutOfMemoryException e) { - logger.error("Exception", e); - context.fail(e); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java index fbffd871664..31f2e4a4912 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java @@ -60,6 +60,15 @@ public BaseRawBatchBuffer(final FragmentContext context, final int fragmentCount this.context = context; } + /** + * Return the fragment count from construction time. + * + * @return the fragment count + */ + protected int getFragmentCount() { + return fragmentCount; + } + @Override public synchronized void enqueue(final RawFragmentBatch batch) throws IOException { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java index cfe5b6b10da..3370380f672 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java @@ -28,9 +28,10 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.memory.AllocatorOwner; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.OutOfMemoryException; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.proto.BitData; import org.apache.drill.exec.proto.BitData.FragmentRecordBatch; @@ -38,6 +39,7 @@ import org.apache.drill.exec.proto.helper.QueryIdHelper; import org.apache.drill.exec.record.RawFragmentBatch; import org.apache.drill.exec.store.LocalSyncableFileSystem; +import org.apache.drill.exec.testing.ExecutionControls; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -82,9 +84,23 @@ private enum SpoolingState { private Path path; private FSDataOutputStream outputStream; - public SpoolingRawBatchBuffer(FragmentContext context, int fragmentCount, int oppositeId, int bufferIndex) throws IOException, OutOfMemoryException { + private final AllocatorOwner allocatorOwner = new AllocatorOwner() { + @Override + public String toString() { + return String.format( + "SpoolingRawBatchBuffer(FragmentContext, fragmentCount = %d, oppositeId = %d, bufferIndex = %d", + getFragmentCount(), oppositeId, bufferIndex); + } + + @Override + public ExecutionControls getExecutionControls() { + return context.getExecutionControls(); + } + }; + + public SpoolingRawBatchBuffer(FragmentContext context, int fragmentCount, int oppositeId, int bufferIndex) { super(context, fragmentCount); - this.allocator = context.getNewChildAllocator(ALLOCATOR_INITIAL_RESERVATION, ALLOCATOR_MAX_RESERVATION, true); + this.allocator = context.newChildAllocator(allocatorOwner, ALLOCATOR_INITIAL_RESERVATION, ALLOCATOR_MAX_RESERVATION, true); this.threshold = context.getConfig().getLong(ExecConstants.SPOOLING_BUFFER_MEMORY); this.oppositeId = oppositeId; this.bufferIndex = bufferIndex; @@ -135,13 +151,14 @@ public boolean isEmpty() { return buffer.size() == 0; } + @Override public void add(RawFragmentBatchWrapper batchWrapper) { buffer.add(batchWrapper); } } private synchronized void setSpoolingState(SpoolingState newState) { - SpoolingState currentState = spoolingState; + final SpoolingState currentState = spoolingState; if (newState == SpoolingState.NOT_SPOOLING || currentState == SpoolingState.STOP_SPOOLING) { return; @@ -216,7 +233,7 @@ protected void enqueueInner(RawFragmentBatch batch) throws IOException { @Override public void kill(FragmentContext context) { - allocator.close(); + DrillAutoCloseables.closeNoChecked(allocator); if (spooler != null) { spooler.terminate(); } @@ -224,12 +241,12 @@ public void kill(FragmentContext context) { @Override protected void upkeep(RawFragmentBatch batch) { - FragmentRecordBatch header = batch.getHeader(); + final FragmentRecordBatch header = batch.getHeader(); if (header.getIsOutOfMemory()) { outOfMemory.set(true); return; } - DrillBuf body = batch.getBody(); + final DrillBuf body = batch.getBody(); if (body != null) { currentSizeInMemory -= body.capacity(); } @@ -253,7 +270,7 @@ public void close() { } } } - allocator.close(); + DrillAutoCloseables.closeNoChecked(allocator); try { if (outputStream != null) { outputStream.close(); @@ -286,6 +303,7 @@ public Spooler(String name) { spoolingQueue = Queues.newLinkedBlockingDeque(); } + @Override public void run() { try { while (shouldContinue) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java index 671deae74ee..0bc5cbce4d0 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java @@ -401,12 +401,9 @@ private void runPhysicalPlan(final PhysicalPlan plan) throws ExecutionSetupExcep logger.debug("Submitting fragments to run."); - // set up the root fragment first so we'll have incoming buffers available. + // Set up the root fragment first so we'll have incoming buffers available. setupRootFragment(rootPlanFragment, work.getRootOperator()); - setupNonRootFragments(planFragments); - drillbitContext.getAllocator().resetFragmentLimits(); // TODO a global effect for this query?!? - moveToState(QueryState.RUNNING, null); logger.debug("Fragments running."); } @@ -789,7 +786,7 @@ protected void processEvent(final StateEvent event) { final Exception exception = event.exception; // TODO Auto-generated method stub - logger.info("State change requested. {} --> {}", state, newState, + logger.info("State change requested. {} --> {}, exception == {}", state, newState, exception); switch (state) { case PENDING: @@ -1011,12 +1008,12 @@ private void setupNonRootFragments(final Collection fragments) thr // if any of the intermediate fragment submissions failed, fail the query final List submissionExceptions = fragmentSubmitFailures.submissionExceptions; if (submissionExceptions.size() > 0) { - Set endpoints = Sets.newHashSet(); - StringBuilder sb = new StringBuilder(); + final Set endpoints = Sets.newHashSet(); + final StringBuilder sb = new StringBuilder(); boolean first = true; for (FragmentSubmitFailures.SubmissionException e : fragmentSubmitFailures.submissionExceptions) { - DrillbitEndpoint endpoint = e.drillbitEndpoint; + final DrillbitEndpoint endpoint = e.drillbitEndpoint; if (endpoints.add(endpoint)) { if (first) { first = false; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ForemanException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ForemanException.java index 32a99ade3c3..c9ed31fd150 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ForemanException.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ForemanException.java @@ -17,41 +17,29 @@ */ package org.apache.drill.exec.work.foreman; -import java.lang.reflect.InvocationTargetException; - import org.apache.drill.common.exceptions.ExecutionSetupException; public class ForemanException extends ExecutionSetupException { private static final long serialVersionUID = -6943409010231014085L; - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ForemanException.class); - - public static ForemanException fromThrowable(String message, Throwable cause) { - Throwable t = cause instanceof InvocationTargetException - ? ((InvocationTargetException)cause).getTargetException() : cause; - if (t instanceof ForemanException) { - return ((ForemanException) t); - } - return new ForemanException(message, t); - } +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ForemanException.class); public ForemanException() { - super(); } - public ForemanException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + public ForemanException(final String message, final Throwable cause, final boolean enableSuppression, + final boolean writableStackTrace) { super(message, cause, enableSuppression, writableStackTrace); } - public ForemanException(String message, Throwable cause) { + public ForemanException(final String message, final Throwable cause) { super(message, cause); } - public ForemanException(String message) { + public ForemanException(final String message) { super(message); } - public ForemanException(Throwable cause) { + public ForemanException(final Throwable cause) { super(cause); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java index 6d03f012b59..c2a2669ca26 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java @@ -338,6 +338,10 @@ private void closeOutResources() { // Say executor was cancelled before setup. Now when executor actually runs, root is not initialized, but this // method is called in finally. So root can be null. if (root != null) { + if (logger.isDebugEnabled()) { + logger.debug(String.format("closing root %s %d", + root.getClass().getName(), System.identityHashCode(root))); + } root.close(); } } catch (final Exception e) { @@ -360,7 +364,7 @@ private void errorStateChange(final FragmentState current, final FragmentState t private synchronized boolean updateState(FragmentState target) { final FragmentState current = fragmentState.get(); - logger.info(fragmentName + ": State change requested {} --> {}", current, target); + logger.info(fragmentName + ": State change requested from {} --> {}", current, target); switch (target) { case CANCELLATION_REQUESTED: switch (current) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java index ad880da0ffa..ff348cba95c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java @@ -30,7 +30,6 @@ * are avialable, a fragment manager will start a fragment executor to run the associated fragment. */ public interface FragmentManager { - /** * Handle the next incoming record batch. * @@ -38,36 +37,43 @@ public interface FragmentManager { * @return True if the fragment has enough incoming data to be able to be run. * @throws FragmentSetupException, IOException */ - public abstract boolean handle(RawFragmentBatch batch) throws FragmentSetupException, IOException; + boolean handle(RawFragmentBatch batch) throws FragmentSetupException, IOException; /** * Get the fragment runner for this incoming fragment. Note, this can only be requested once. * * @return */ - public abstract FragmentExecutor getRunnable(); + FragmentExecutor getRunnable(); + + void cancel(); - public abstract void cancel(); + /** + * Find out if the FragmentManager has been cancelled. + * + * @return true if the FragmentManager has been cancelled. + */ + boolean isCancelled(); /** * If the executor is paused (for testing), this method should unpause the executor. This method should handle * multiple calls. */ - public abstract void unpause(); + void unpause(); - public boolean isWaiting(); + boolean isWaiting(); - public abstract FragmentHandle getHandle(); + FragmentHandle getHandle(); - public abstract FragmentContext getFragmentContext(); + FragmentContext getFragmentContext(); - public abstract void addConnection(RemoteConnection connection); + void addConnection(RemoteConnection connection); - public void receivingFragmentFinished(final FragmentHandle handle); + void receivingFragmentFinished(final FragmentHandle handle); /** * Sets autoRead property on all connections * @param autoRead */ - public abstract void setAutoRead(boolean autoRead); -} \ No newline at end of file + void setAutoRead(boolean autoRead); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java index 3fc757c5dfe..9378e51d4ea 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java @@ -39,7 +39,7 @@ */ // TODO a lot of this is the same as RootFragmentManager public class NonRootFragmentManager implements FragmentManager { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NonRootFragmentManager.class); + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NonRootFragmentManager.class); private final IncomingBuffers buffers; private final FragmentExecutor runner; @@ -97,15 +97,15 @@ public void receivingFragmentFinished(final FragmentHandle handle) { runner.receivingFragmentFinished(handle); } - /* (non-Javadoc) - * @see org.apache.drill.exec.work.fragment.FragmentHandler#cancel() - */ @Override - public void cancel() { - synchronized(this) { - cancel = true; - runner.cancel(); - } + public synchronized void cancel() { + cancel = true; + runner.cancel(); + } + + @Override + public boolean isCancelled() { + return cancel; } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java index f4f76dd91c4..0713398b441 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java @@ -30,7 +30,7 @@ // TODO a lot of this is the same as NonRootFragmentManager public class RootFragmentManager implements FragmentManager { -// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootFragmentManager.class); + // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootFragmentManager.class); private final IncomingBuffers buffers; private final FragmentExecutor runner; @@ -70,6 +70,11 @@ public void cancel() { runner.cancel(); } + @Override + public boolean isCancelled() { + return cancel; + } + @Override public void unpause() { runner.unpause(); diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java index d428920ecdc..ed034ef0449 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java +++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java @@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger; import com.google.common.io.Files; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.exceptions.UserException; import org.apache.drill.exec.ExecConstants; @@ -34,8 +35,8 @@ import org.apache.drill.exec.client.DrillClient; import org.apache.drill.exec.exception.SchemaChangeException; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; import org.apache.drill.exec.proto.UserBitShared; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.proto.UserBitShared.QueryId; import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState; import org.apache.drill.exec.proto.UserBitShared.QueryType; @@ -166,7 +167,7 @@ private static void resetClientAndBit() throws Exception{ } private static void openClient() throws Exception { - allocator = new TopLevelAllocator(config); + allocator = new RootAllocator(config); if (config.hasPath(ENABLE_FULL_CACHE) && config.getBoolean(ENABLE_FULL_CACHE)) { serviceSet = RemoteServiceSet.getServiceSetWithFullCache(config, allocator); } else { @@ -227,6 +228,26 @@ public static TestBuilder testBuilder() { return new TestBuilder(allocator); } + /** + * Utility function that can be used in tests to verify the state of drillbit + * allocators. + */ + public static void verifyAllocators() { + if (bits != null) { + for(Drillbit bit : bits) { + if (bit != null) { + final DrillbitContext drillbitContext = bit.getContext(); + final BufferAllocator bufferAllocator = drillbitContext.getAllocator(); + if (!(bufferAllocator instanceof RootAllocator)) { + throw new IllegalStateException("The DrillbitContext's allocator is not a RootAllocator"); + } + final RootAllocator rootAllocator = (RootAllocator) bufferAllocator; + rootAllocator.verify(); + } + } + } + } + @AfterClass public static void closeClient() throws IOException, InterruptedException { if (client != null) { @@ -245,7 +266,7 @@ public static void closeClient() throws IOException, InterruptedException { serviceSet.close(); } if (allocator != null) { - allocator.close(); + DrillAutoCloseables.closeNoChecked(allocator); } } diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java index da033f7d002..9f5deaa218f 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java +++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java @@ -22,14 +22,14 @@ import mockit.Mocked; import mockit.NonStrictExpectations; + import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.jdbc.SimpleCalciteSchema; - import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.util.TestTools; import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.planner.physical.PlannerSettings; @@ -53,7 +53,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.io.Resources; -public class PlanningBase extends ExecTest{ +public class PlanningBase extends ExecTest { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlanningBase.class); @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(10000); @@ -63,7 +63,11 @@ public class PlanningBase extends ExecTest{ @Mocked QueryContext context; - TopLevelAllocator allocator = new TopLevelAllocator(); + final RootAllocator allocator; + + public PlanningBase() throws Exception { + allocator = new RootAllocator(config); + } protected void testSqlPlanFromFile(String file) throws Exception { testSqlPlan(getFile(file)); @@ -91,6 +95,8 @@ protected void testSqlPlan(String sqlCommands) throws Exception { result = allocator; dbContext.getConfig(); result = config; + dbContext.getAllocator(); + result = new RootAllocator(config); dbContext.getOptionManager(); result = systemOptions; dbContext.getPersistentStoreProvider(); @@ -141,7 +147,6 @@ protected void testSqlPlan(String sqlCommands) throws Exception { DrillSqlWorker worker = new DrillSqlWorker(context); PhysicalPlan p = worker.getPlan(sql); } - } protected String getFile(String resource) throws IOException{ diff --git a/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java b/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java index e218d6ce615..7053d931642 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java +++ b/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java @@ -28,6 +28,7 @@ import org.apache.drill.exec.client.DrillClient; import org.apache.drill.exec.client.PrintingResultsListener; import org.apache.drill.exec.client.QuerySubmitter.Format; +import org.apache.drill.exec.memory.OutOfMemoryException; import org.apache.drill.exec.proto.UserBitShared.QueryType; import org.apache.drill.exec.rpc.RpcException; import org.apache.drill.exec.rpc.user.QueryDataBatch; @@ -56,7 +57,7 @@ private QueryTestUtil() { * @throws RpcException if there is a problem setting up the client */ public static DrillClient createClient(final DrillConfig drillConfig, final RemoteServiceSet remoteServiceSet, - final int maxWidth, final Properties props) throws RpcException { + final int maxWidth, final Properties props) throws RpcException, OutOfMemoryException { final DrillClient drillClient = new DrillClient(drillConfig, remoteServiceSet.getCoordinator()); drillClient.connect(props); diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestAllocationException.java b/exec/java-exec/src/test/java/org/apache/drill/TestAllocationException.java index 10fd5da5e45..939039c2ca1 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/TestAllocationException.java +++ b/exec/java-exec/src/test/java/org/apache/drill/TestAllocationException.java @@ -18,14 +18,17 @@ package org.apache.drill; import org.apache.drill.common.exceptions.UserException; +import org.apache.drill.exec.memory.BaseAllocator; import org.apache.drill.exec.memory.OutOfMemoryRuntimeException; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.proto.UserBitShared.DrillPBError; import org.apache.drill.exec.testing.Controls; import org.apache.drill.exec.testing.ControlsInjectionUtil; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; + import org.junit.Test; /** @@ -33,31 +36,30 @@ * be propagated downstream. Make sure the proper "memory error" message is sent to the client. */ public class TestAllocationException extends BaseTestQuery { - private static final String SINGLE_MODE = "ALTER SESSION SET `planner.disable_exchanges` = true"; - private void testWithException(final String fileName) throws Exception{ + private void testWithException(final String fileName) throws Exception { test(SINGLE_MODE); final String controls = Controls.newBuilder() - .addException(TopLevelAllocator.class, - TopLevelAllocator.CHILD_BUFFER_INJECTION_SITE, + .addException(BaseAllocator.class, + RootAllocator.CHILD_BUFFER_INJECTION_SITE, OutOfMemoryRuntimeException.class, 200, 1 ).build(); ControlsInjectionUtil.setControls(client, controls); - String query = getFile(fileName); + final String query = getFile(fileName); try { test(query); fail("The query should have failed!"); - } catch(UserException uex) { - DrillPBError error = uex.getOrCreatePBError(false); + } catch(final UserException uex) { + final DrillPBError error = uex.getOrCreatePBError(false); assertEquals(DrillPBError.ErrorType.RESOURCE, error.getErrorType()); assertTrue("Error message isn't related to memory error", - uex.getMessage().contains(UserException.MEMORY_ERROR_MSG)); + uex.getMessage().contains(UserException.MEMORY_ERROR_MSG)); } } diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java index 22471c8abbd..6d2fbf02ce5 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java +++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java @@ -17,12 +17,18 @@ */ package org.apache.drill; +import org.junit.After; import org.junit.Ignore; import org.junit.Test; public class TestTpchLimit0 extends BaseTestQuery{ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchLimit0.class); + @After + public void checkForLeaks() { + verifyAllocators(); + } + private void testLimitZero(String fileName) throws Exception { String query = getFile(fileName); query = "ALTER SESSION SET `planner.slice_target` = 1; select * from \n(" + query.replace(";", ")xyz limit 0;"); diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java index 707ea78a8ce..737e1524b64 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java +++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java @@ -24,131 +24,133 @@ import org.junit.rules.TestRule; -public class TestTpchPlanning extends PlanningBase{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchPlanning.class); +public class TestTpchPlanning extends PlanningBase { +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchPlanning.class); @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(50000); + public TestTpchPlanning() throws Exception { + } + @Test - public void tpch01() throws Exception{ + public void tpch01() throws Exception { testSqlPlanFromFile("queries/tpch/01.sql"); } @Test @Ignore // DRILL-512 - public void tpch02() throws Exception{ + public void tpch02() throws Exception { testSqlPlanFromFile("queries/tpch/02.sql"); } @Test - public void tpch03() throws Exception{ + public void tpch03() throws Exception { testSqlPlanFromFile("queries/tpch/03.sql"); } @Test - public void tpch04() throws Exception{ + public void tpch04() throws Exception { testSqlPlanFromFile("queries/tpch/04.sql"); } @Test - public void tpch05() throws Exception{ + public void tpch05() throws Exception { testSqlPlanFromFile("queries/tpch/05.sql"); } @Test - public void tpch06() throws Exception{ + public void tpch06() throws Exception { testSqlPlanFromFile("queries/tpch/06.sql"); } @Test - public void tpch07() throws Exception{ + public void tpch07() throws Exception { testSqlPlanFromFile("queries/tpch/07.sql"); } @Test @Ignore // cannot plan exception (was DRILL-516) - public void tpch08() throws Exception{ + public void tpch08() throws Exception { testSqlPlanFromFile("queries/tpch/08.sql"); } @Test @Ignore // cannot plan exception (was DRILL-516) - public void tpch09() throws Exception{ + public void tpch09() throws Exception { testSqlPlanFromFile("queries/tpch/09.sql"); } @Test - public void tpch10() throws Exception{ + public void tpch10() throws Exception { testSqlPlanFromFile("queries/tpch/10.sql"); } @Test @Ignore // cartesion problem - public void tpch11() throws Exception{ + public void tpch11() throws Exception { testSqlPlanFromFile("queries/tpch/11.sql"); } @Test - public void tpch12() throws Exception{ + public void tpch12() throws Exception { testSqlPlanFromFile("queries/tpch/12.sql"); } @Test @Ignore // sporadic failures when part of the full build. - public void tpch13() throws Exception{ + public void tpch13() throws Exception { testSqlPlanFromFile("queries/tpch/13.sql"); } @Test - public void tpch14() throws Exception{ + public void tpch14() throws Exception { testSqlPlanFromFile("queries/tpch/14.sql"); } @Test @Ignore // requires views. - public void tpch15() throws Exception{ + public void tpch15() throws Exception { testSqlPlanFromFile("queries/tpch/15.sql"); } @Test @Ignore // invalid plan, due to Nulls value NOT IN sub-q - public void tpch16() throws Exception{ + public void tpch16() throws Exception { testSqlPlanFromFile("queries/tpch/16.sql"); } @Test @Ignore // DRILL-517 - public void tpch17() throws Exception{ + public void tpch17() throws Exception { testSqlPlanFromFile("queries/tpch/17.sql"); } @Test - public void tpch18() throws Exception{ + public void tpch18() throws Exception { testSqlPlanFromFile("queries/tpch/18.sql"); } @Test @Ignore // DRILL-519 - public void tpch19() throws Exception{ + public void tpch19() throws Exception { testSqlPlanFromFile("queries/tpch/19.sql"); } @Test @Ignore // DRILL-517 - public void tpch20() throws Exception{ + public void tpch20() throws Exception { testSqlPlanFromFile("queries/tpch/20.sql"); } @Test @Ignore // DRILL-519 - public void tpch21() throws Exception{ + public void tpch21() throws Exception { testSqlPlanFromFile("queries/tpch/21.sql"); } @Test @Ignore // DRILL-518 - public void tpch22() throws Exception{ + public void tpch22() throws Exception { testSqlPlanFromFile("queries/tpch/22.sql"); } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java index 7c58b19f5b6..b3a9f80746c 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java @@ -30,7 +30,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -58,17 +58,16 @@ * which will produce a dump file. The dump file will be input into DumpCat to test query mode and batch mode. */ -public class DumpCatTest extends ExecTest{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DumpCatTest.class); - DrillConfig c = DrillConfig.create(); +public class DumpCatTest extends ExecTest { +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DumpCatTest.class); + private final DrillConfig c = DrillConfig.create(); @Test public void testDumpCat(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable { - new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); @@ -127,5 +126,4 @@ public void testDumpCat(@Injectable final DrillbitContext bitContext, @Injectabl input.close(); } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java index 5e50189d0d5..4e0f5be05b0 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java @@ -42,8 +42,7 @@ import org.apache.drill.common.types.Types; import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.physical.impl.project.Projector; import org.apache.drill.exec.record.MaterializedField; import org.apache.drill.exec.record.RecordBatch; @@ -54,7 +53,7 @@ import org.junit.Test; public class ExpressionTest extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionTest.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionTest.class); DrillConfig c = DrillConfig.create(); FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c); @@ -87,7 +86,7 @@ public void testSpecial(final @Injectable RecordBatch batch, @Injectable ValueVe batch.getValueAccessorById(IntVector.class, tfid.getFieldIds()); result = wrapper; wrapper.getValueVector(); - result = new IntVector(MaterializedField.create("result", type), new TopLevelAllocator(0)); + result = new IntVector(MaterializedField.create("result", type), new RootAllocator(DrillConfig.create())); } }; diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestByteComparisonFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestByteComparisonFunctions.java index 1e9a47c995e..7a40b565826 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestByteComparisonFunctions.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestByteComparisonFunctions.java @@ -19,11 +19,12 @@ import static org.junit.Assert.assertTrue; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers; import org.apache.drill.exec.expr.holders.VarCharHolder; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.vector.ValueHolderHelper; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -32,16 +33,16 @@ public class TestByteComparisonFunctions extends ExecTest{ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestByteComparisonFunctions.class); - static TopLevelAllocator allocator; + static RootAllocator allocator; static VarCharHolder hello; static VarCharHolder goodbye; static VarCharHolder helloLong; static VarCharHolder goodbyeLong; @BeforeClass - public static void setup(){ - DrillConfig c= DrillConfig.create(); - allocator = new TopLevelAllocator(c); + public static void setup() throws Exception { + final DrillConfig c = DrillConfig.create(); + allocator = new RootAllocator(c); hello = ValueHolderHelper.getVarCharHolder(allocator, "hello"); goodbye = ValueHolderHelper.getVarCharHolder(allocator, "goodbye"); helloLong = ValueHolderHelper.getVarCharHolder(allocator, "hellomyfriend"); @@ -54,7 +55,7 @@ public static void teardown(){ helloLong.buffer.release(); goodbye.buffer.release(); goodbyeLong.buffer.release(); - allocator.close(); + DrillAutoCloseables.closeNoChecked(allocator); } @Test diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java index 3ba8743e1d1..d10d0987972 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java @@ -17,15 +17,11 @@ */ package org.apache.drill.exec.fn.impl; - import org.apache.drill.BaseTestQuery; import org.apache.drill.common.util.FileUtils; import org.junit.Test; -import java.util.Date; - public class TestCastFunctions extends BaseTestQuery { - @Test public void testVarbinaryToDate() throws Exception { testBuilder() @@ -51,5 +47,4 @@ public void testImplicitCastStringToBoolean() throws Exception { .baselineValues(false) .build().run(); } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java index 4f06a9d5bf0..529f8ceb14f 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java @@ -30,7 +30,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -52,19 +52,16 @@ public class TestMathFunctions extends ExecTest { - - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMathFunctions.class); - DrillConfig c = DrillConfig.create(); - +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMathFunctions.class); + final DrillConfig c = DrillConfig.create(); @Test public void testBasicMathFunctions(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable { - new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); bitContext.getConfig(); result = c; + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); }}; diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java index 880184eee1e..b120ef6e8ce 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java @@ -29,7 +29,7 @@ import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -50,17 +50,16 @@ import com.google.common.io.Resources; public class TestNewMathFunctions { +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestNewMathFunctions.class); - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestNewMathFunctions.class); - - DrillConfig c = DrillConfig.create(); + final DrillConfig c = DrillConfig.create(); PhysicalPlanReader reader; FunctionImplementationRegistry registry; FragmentContext context; public Object[] getRunResult(SimpleRootExec exec) { int size = 0; - for (ValueVector v : exec) { + for (@SuppressWarnings("unused") ValueVector v : exec) { size++; } @@ -81,9 +80,9 @@ public void runTest(@Injectable final DrillbitContext bitContext, new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); }}; @@ -147,5 +146,4 @@ public void testIsNumeric(@Injectable final DrillbitContext bitContext, Object [] expected = new Object[] {1, 1, 1, 0}; runTest(bitContext, connection, expected, "functions/testIsNumericFunction.json"); } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.java index 73c750833b2..7164c0d6386 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.java @@ -29,7 +29,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -50,24 +50,19 @@ import com.google.common.io.Files; public class TestRepeatedFunction extends ExecTest{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestRepeatedFunction.class); - DrillConfig c = DrillConfig.create(); - + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestRepeatedFunction.class); + final DrillConfig c = DrillConfig.create(); @Test public void testRepeated(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{ -// System.out.println(System.getProperty("java.class.path")); - - new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); }}; - PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance()); PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/physical_repeated_1.json"), Charsets.UTF_8)); FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c); @@ -108,7 +103,5 @@ public void testRepeated(@Injectable final DrillbitContext bitContext, @Injectab throw context.getFailureCause(); } assertTrue(!context.isFailed()); - } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java index 0b8314c0a34..c8e7ebe9483 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java @@ -18,8 +18,13 @@ package org.apache.drill.exec.memory; +import com.google.common.base.Charsets; +import com.google.common.io.Files; import static org.junit.Assert.fail; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import io.netty.buffer.DrillBuf; @@ -27,6 +32,8 @@ import java.util.List; import java.util.Properties; +import io.netty.buffer.UnsafeDirectLittleEndian; + import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.util.FileUtils; import org.apache.drill.exec.ExecConstants; @@ -47,12 +54,10 @@ import org.apache.drill.exec.store.StoragePluginRegistry; import org.junit.Test; -import com.google.common.base.Charsets; -import com.google.common.io.Files; - public class TestAllocators { - private static final Properties TEST_CONFIGURATIONS = new Properties() { + private static final long serialVersionUID = 1L; + { put(ExecConstants.TOP_LEVEL_MAX_ALLOC, "14000000"); put(ExecConstants.ENABLE_FRAGMENT_MEMORY_LIMIT, "true"); @@ -60,7 +65,7 @@ public class TestAllocators { } }; - private final static String planFile="/physical_allocator_test.json"; + private final static String planFile = "/physical_allocator_test.json"; @Test public void testAllocators() throws Exception { @@ -70,33 +75,35 @@ public void testAllocators() throws Exception { final Drillbit bit = new Drillbit(config, serviceSet); bit.run(); final DrillbitContext bitContext = bit.getContext(); - FunctionImplementationRegistry functionRegistry = bitContext.getFunctionImplementationRegistry(); - StoragePluginRegistry storageRegistry = new StoragePluginRegistry(bitContext); + final FunctionImplementationRegistry functionRegistry = bitContext.getFunctionImplementationRegistry(); + final StoragePluginRegistry storageRegistry = new StoragePluginRegistry(bitContext); // Create a few Fragment Contexts - BitControl.PlanFragment.Builder pfBuilder1=BitControl.PlanFragment.newBuilder(); + final BitControl.PlanFragment.Builder pfBuilder1 = BitControl.PlanFragment.newBuilder(); pfBuilder1.setMemInitial(1500000); - BitControl.PlanFragment pf1=pfBuilder1.build(); - BitControl.PlanFragment.Builder pfBuilder2=BitControl.PlanFragment.newBuilder(); + final BitControl.PlanFragment pf1 = pfBuilder1.build(); + final BitControl.PlanFragment.Builder pfBuilder2 = BitControl.PlanFragment.newBuilder(); pfBuilder2.setMemInitial(500000); - BitControl.PlanFragment pf2=pfBuilder1.build(); + final BitControl.PlanFragment pf2 = pfBuilder1.build(); - FragmentContext fragmentContext1 = new FragmentContext(bitContext, pf1, null, functionRegistry); - FragmentContext fragmentContext2 = new FragmentContext(bitContext, pf2, null, functionRegistry); + final FragmentContext fragmentContext1 = new FragmentContext(bitContext, pf1, null, functionRegistry); + final FragmentContext fragmentContext2 = new FragmentContext(bitContext, pf2, null, functionRegistry); // Get a few physical operators. Easiest way is to read a physical plan. - PhysicalPlanReader planReader = new PhysicalPlanReader(config, config.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), storageRegistry); - PhysicalPlan plan = planReader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(planFile), Charsets.UTF_8)); - List physicalOperators = plan.getSortedOperators(); - Iterator physicalOperatorIterator = physicalOperators.iterator(); - - PhysicalOperator physicalOperator1 = physicalOperatorIterator.next(); - PhysicalOperator physicalOperator2 = physicalOperatorIterator.next(); - PhysicalOperator physicalOperator3 = physicalOperatorIterator.next(); - PhysicalOperator physicalOperator4 = physicalOperatorIterator.next(); - PhysicalOperator physicalOperator5 = physicalOperatorIterator.next(); - PhysicalOperator physicalOperator6 = physicalOperatorIterator.next(); + final PhysicalPlanReader planReader = new PhysicalPlanReader(config, config.getMapper(), + CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), storageRegistry); + final PhysicalPlan plan = planReader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(planFile), + Charsets.UTF_8)); + final List physicalOperators = plan.getSortedOperators(); + final Iterator physicalOperatorIterator = physicalOperators.iterator(); + + final PhysicalOperator physicalOperator1 = physicalOperatorIterator.next(); + final PhysicalOperator physicalOperator2 = physicalOperatorIterator.next(); + final PhysicalOperator physicalOperator3 = physicalOperatorIterator.next(); + final PhysicalOperator physicalOperator4 = physicalOperatorIterator.next(); + final PhysicalOperator physicalOperator5 = physicalOperatorIterator.next(); + final PhysicalOperator physicalOperator6 = physicalOperatorIterator.next(); // Create some bogus Operator profile defs and stats to create operator contexts OpProfileDef def; @@ -107,54 +114,55 @@ public void testAllocators() throws Exception { OperatorContext.getChildCount(physicalOperator1)); stats = fragmentContext1.getStats().getOperatorStats(def, fragmentContext1.getAllocator()); - // Add a couple of Operator Contexts // Initial allocation = 1000000 bytes for all operators - OperatorContext oContext11 = fragmentContext1.newOperatorContext(physicalOperator1, true); - DrillBuf b11=oContext11.getAllocator().buffer(1000000); + final OperatorContext oContext11 = fragmentContext1.newOperatorContext(physicalOperator1, true); + final DrillBuf b11=oContext11.getAllocator().buffer(1000000); - OperatorContext oContext12 = fragmentContext1.newOperatorContext(physicalOperator2, stats, true); - DrillBuf b12=oContext12.getAllocator().buffer(500000); + final OperatorContext oContext12 = fragmentContext1.newOperatorContext(physicalOperator2, stats, true); + final DrillBuf b12=oContext12.getAllocator().buffer(500000); - OperatorContext oContext21 = fragmentContext1.newOperatorContext(physicalOperator3, true); + final OperatorContext oContext21 = fragmentContext1.newOperatorContext(physicalOperator3, true); def = new OpProfileDef(physicalOperator4.getOperatorId(), UserBitShared.CoreOperatorType.TEXT_WRITER_VALUE, OperatorContext.getChildCount(physicalOperator4)); stats = fragmentContext2.getStats().getOperatorStats(def, fragmentContext2.getAllocator()); - OperatorContext oContext22 = fragmentContext2.newOperatorContext(physicalOperator4, stats, true); - DrillBuf b22=oContext22.getAllocator().buffer(2000000); + final OperatorContext oContext22 = fragmentContext2.newOperatorContext(physicalOperator4, stats, true); + final DrillBuf b22=oContext22.getAllocator().buffer(2000000); // New Fragment begins - BitControl.PlanFragment.Builder pfBuilder3=BitControl.PlanFragment.newBuilder(); + final BitControl.PlanFragment.Builder pfBuilder3 = BitControl.PlanFragment.newBuilder(); pfBuilder3.setMemInitial(1000000); BitControl.PlanFragment pf3=pfBuilder3.build(); - FragmentContext fragmentContext3 = new FragmentContext(bitContext, pf3, null, functionRegistry); + final FragmentContext fragmentContext3 = new FragmentContext(bitContext, pf3, null, functionRegistry); // New fragment starts an operator that allocates an amount within the limit def = new OpProfileDef(physicalOperator5.getOperatorId(), UserBitShared.CoreOperatorType.UNION_VALUE, OperatorContext.getChildCount(physicalOperator5)); stats = fragmentContext3.getStats().getOperatorStats(def, fragmentContext3.getAllocator()); - OperatorContext oContext31 = fragmentContext3.newOperatorContext(physicalOperator5, stats, true); + final OperatorContext oContext31 = fragmentContext3.newOperatorContext(physicalOperator5, stats, true); - DrillBuf b31a = oContext31.getAllocator().buffer(200000); + final DrillBuf b31a = oContext31.getAllocator().buffer(200000); - //Previously running operator completes + // Previously running operator completes b22.release(); ((AutoCloseable) oContext22).close(); // Fragment 3 asks for more and fails + boolean outOfMem = false; try { - oContext31.getAllocator().buffer(4400000); + oContext31.getAllocator().buffer(44000000); fail("Fragment 3 should fail to allocate buffer"); } catch (OutOfMemoryRuntimeException e) { - // expected + outOfMem = true; // Expected. } + assertTrue(outOfMem); // Operator is Exempt from Fragment limits. Fragment 3 asks for more and succeeds - OperatorContext oContext32 = fragmentContext3.newOperatorContext(physicalOperator6, false); + final OperatorContext oContext32 = fragmentContext3.newOperatorContext(physicalOperator6, false); try { - DrillBuf b32 = oContext32.getAllocator().buffer(4400000); + final DrillBuf b32 = oContext32.getAllocator().buffer(4400000); b32.release(); } catch (OutOfMemoryRuntimeException e) { fail("Fragment 3 failed to allocate buffer"); @@ -176,6 +184,12 @@ public void testAllocators() throws Exception { bit.close(); serviceSet.close(); + +/* + // ---------------------------------------- DEBUG ---------------------------------- + assertEquals(0, UnsafeDirectLittleEndian.getBufferCount()); + // ---------------------------------------- DEBUG ---------------------------------- +*/ } private void closeOp(OperatorContext c) throws Exception { diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java new file mode 100644 index 00000000000..bad14a4372b --- /dev/null +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java @@ -0,0 +1,645 @@ +/** + * 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.memory; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import io.netty.buffer.DrillBuf; + +import org.apache.drill.exec.server.options.OptionManager; +import org.apache.drill.exec.testing.ExecutionControls; +import org.apache.drill.exec.util.Pointer; +import org.junit.Ignore; +import org.junit.Test; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.Mockito; + +public class TestBaseAllocator { + // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBaseAllocator.class); + + private final static int MAX_ALLOCATION = 8 * 1024; + +/* + // ---------------------------------------- DEBUG ----------------------------------- + + @After + public void checkBuffers() { + final int bufferCount = UnsafeDirectLittleEndian.getBufferCount(); + if (bufferCount != 0) { + UnsafeDirectLittleEndian.logBuffers(logger); + UnsafeDirectLittleEndian.releaseBuffers(); + } + + assertEquals(0, bufferCount); + } + +// @AfterClass +// public static void dumpBuffers() { +// UnsafeDirectLittleEndian.logBuffers(logger); +// } + + // ---------------------------------------- DEBUG ------------------------------------ +*/ + + // Concoct ExecutionControls that won't try to inject anything. + @Mock private static final OptionManager optionManager = Mockito.mock(OptionManager.class); + static { + Mockito.when(optionManager.getOption(Matchers.anyString())) + .thenReturn(null); + } + + @Mock private static final ExecutionControls executionControls = new ExecutionControls(optionManager, null); + + private final static class NamedOwner implements AllocatorOwner { + private final String name; + + public NamedOwner(final String name) { + this.name = name; + } + + @Override + public String toString() { + return name; + } + + @Override + public ExecutionControls getExecutionControls() { + return executionControls; + } + } + + @Test + public void test_privateMax() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("noLimits"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_LOCAL_MAX, 0, MAX_ALLOCATION, 0)) { + final DrillBuf drillBuf1 = rootAllocator.buffer(MAX_ALLOCATION / 2); + assertNotNull("allocation failed", drillBuf1); + + try(final BufferAllocator childAllocator = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + final DrillBuf drillBuf2 = childAllocator.buffer(MAX_ALLOCATION / 2); + assertNotNull("allocation failed", drillBuf2); + drillBuf2.release(); + } + + drillBuf1.release(); + } + } + + @Test(expected=IllegalStateException.class) + public void testRootAllocator_closeWithOutstanding() throws Exception { + try { + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + final DrillBuf drillBuf = rootAllocator.buffer(512); + assertNotNull("allocation failed", drillBuf); + } + } finally { + /* + * We expect there to be one unreleased underlying buffer because we're closing + * without releasing it. + */ +/* + // ------------------------------- DEBUG --------------------------------- + final int bufferCount = UnsafeDirectLittleEndian.getBufferCount(); + UnsafeDirectLittleEndian.releaseBuffers(); + assertEquals(1, bufferCount); + // ------------------------------- DEBUG --------------------------------- +*/ + } + } + + @Test + public void testRootAllocator_getEmpty() throws Exception { + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + final DrillBuf drillBuf = rootAllocator.buffer(0); + assertNotNull("allocation failed", drillBuf); + assertEquals("capacity was non-zero", 0, drillBuf.capacity()); + drillBuf.release(); + } + } + + @Ignore // TODO(DRILL-2740) + @Test(expected = IllegalStateException.class) + public void testAllocator_unreleasedEmpty() throws Exception { + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + @SuppressWarnings("unused") + final DrillBuf drillBuf = rootAllocator.buffer(0); + } + } + + @Test + public void testAllocator_transferOwnership() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("changeOwnership"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + final BufferAllocator childAllocator1 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + final BufferAllocator childAllocator2 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + + final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 4); + rootAllocator.verify(); + final boolean allocationFit = childAllocator2.takeOwnership(drillBuf1); + rootAllocator.verify(); + assertTrue(allocationFit); + + childAllocator1.close(); + rootAllocator.verify(); + + drillBuf1.release(); + childAllocator2.close(); + } + } + + @Test + public void testAllocator_shareOwnership() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("shareOwnership"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + final BufferAllocator childAllocator1 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + final BufferAllocator childAllocator2 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 4); + rootAllocator.verify(); + final Pointer pDrillBuf = new Pointer<>(); + boolean allocationFit; + + allocationFit = childAllocator2.shareOwnership(drillBuf1, pDrillBuf); + assertTrue(allocationFit); + rootAllocator.verify(); + final DrillBuf drillBuf2 = pDrillBuf.value; + assertNotNull(drillBuf2); + assertNotEquals(drillBuf2, drillBuf1); + + drillBuf1.release(); + rootAllocator.verify(); + childAllocator1.close(); + rootAllocator.verify(); + + final BufferAllocator childAllocator3 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + allocationFit = childAllocator3.shareOwnership(drillBuf2, pDrillBuf); + assertTrue(allocationFit); + final DrillBuf drillBuf3 = pDrillBuf.value; + assertNotNull(drillBuf3); + assertNotEquals(drillBuf3, drillBuf1); + assertNotEquals(drillBuf3, drillBuf2); + rootAllocator.verify(); + + drillBuf2.release(); + rootAllocator.verify(); + childAllocator2.close(); + rootAllocator.verify(); + + drillBuf3.release(); + rootAllocator.verify(); + childAllocator3.close(); + } + } + + @Test + public void testRootAllocator_createChildAndUse() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("createChildAndUse"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + try(final BufferAllocator childAllocator = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + final DrillBuf drillBuf = childAllocator.buffer(512); + assertNotNull("allocation failed", drillBuf); + drillBuf.release(); + } + } + } + + @Test(expected=IllegalStateException.class) + public void testRootAllocator_createChildDontClose() throws Exception { + try { + final AllocatorOwner allocatorOwner = new NamedOwner("createChildDontClose"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + final BufferAllocator childAllocator = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + final DrillBuf drillBuf = childAllocator.buffer(512); + assertNotNull("allocation failed", drillBuf); + } + } finally { + /* + * We expect one underlying buffer because we closed a child allocator without + * releasing the buffer allocated from it. + */ +/* + // ------------------------------- DEBUG --------------------------------- + final int bufferCount = UnsafeDirectLittleEndian.getBufferCount(); + UnsafeDirectLittleEndian.releaseBuffers(); + assertEquals(1, bufferCount); + // ------------------------------- DEBUG --------------------------------- +*/ + } + } + + private static void allocateAndFree(final BufferAllocator allocator) { + final DrillBuf drillBuf = allocator.buffer(512); + assertNotNull("allocation failed", drillBuf); + drillBuf.release(); + + final DrillBuf drillBuf2 = allocator.buffer(MAX_ALLOCATION); + assertNotNull("allocation failed", drillBuf2); + drillBuf2.release(); + + final int nBufs = 8; + final DrillBuf[] drillBufs = new DrillBuf[nBufs]; + for(int i = 0; i < drillBufs.length; ++i) { + DrillBuf drillBufi = allocator.buffer(MAX_ALLOCATION / nBufs); + assertNotNull("allocation failed", drillBufi); + drillBufs[i] = drillBufi; + } + for(DrillBuf drillBufi : drillBufs) { + drillBufi.release(); + } + } + + @Test + public void testAllocator_manyAllocations() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("manyAllocations"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + try(final BufferAllocator childAllocator = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + allocateAndFree(childAllocator); + } + } + } + + @Test + public void testAllocator_overAllocate() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("overAllocate"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + try(final BufferAllocator childAllocator = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + allocateAndFree(childAllocator); + + try { + childAllocator.buffer(MAX_ALLOCATION + 1); + fail("allocated memory beyond max allowed"); + } catch(OutOfMemoryRuntimeException e) { + // expected + } + } + } + } + + @Test + public void testAllocator_overAllocateParent() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("overAllocateParent"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + try(final BufferAllocator childAllocator = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + final DrillBuf drillBuf1 = rootAllocator.buffer(MAX_ALLOCATION / 2); + assertNotNull("allocation failed", drillBuf1); + final DrillBuf drillBuf2 = childAllocator.buffer(MAX_ALLOCATION / 2); + assertNotNull("allocation failed", drillBuf2); + + try { + childAllocator.buffer(MAX_ALLOCATION / 4); + fail("allocated memory beyond max allowed"); + } catch(OutOfMemoryRuntimeException e) { + // expected + } + + drillBuf1.release(); + drillBuf2.release(); + } + } + } + + private static void testAllocator_sliceUpBufferAndRelease( + final RootAllocator rootAllocator, final BufferAllocator bufferAllocator) { + final DrillBuf drillBuf1 = bufferAllocator.buffer(MAX_ALLOCATION / 2); + rootAllocator.verify(); + + final DrillBuf drillBuf2 = drillBuf1.slice(16, drillBuf1.capacity() - 32); + rootAllocator.verify(); + final DrillBuf drillBuf3 = drillBuf2.slice(16, drillBuf2.capacity() - 32); + rootAllocator.verify(); + @SuppressWarnings("unused") + final DrillBuf drillBuf4 = drillBuf3.slice(16, drillBuf3.capacity() - 32); + rootAllocator.verify(); + + drillBuf3.release(); // since they share refcounts, one is enough to release them all + rootAllocator.verify(); + } + + @Test + public void testAllocator_createSlices() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("createSlices"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + testAllocator_sliceUpBufferAndRelease(rootAllocator, rootAllocator); + + try(final BufferAllocator childAllocator = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + testAllocator_sliceUpBufferAndRelease(rootAllocator, childAllocator); + } + rootAllocator.verify(); + + testAllocator_sliceUpBufferAndRelease(rootAllocator, rootAllocator); + + try(final BufferAllocator childAllocator = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + try(final BufferAllocator childAllocator2 = + childAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + final DrillBuf drillBuf1 = childAllocator2.buffer(MAX_ALLOCATION / 8); + @SuppressWarnings("unused") + final DrillBuf drillBuf2 = drillBuf1.slice(MAX_ALLOCATION / 16, MAX_ALLOCATION / 16); + testAllocator_sliceUpBufferAndRelease(rootAllocator, childAllocator); + drillBuf1.release(); + rootAllocator.verify(); + } + rootAllocator.verify(); + + testAllocator_sliceUpBufferAndRelease(rootAllocator, childAllocator); + } + rootAllocator.verify(); + } + } + + @Test + public void testAllocator_sliceRanges() throws Exception { +// final AllocatorOwner allocatorOwner = new NamedOwner("sliceRanges"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + // Populate a buffer with byte values corresponding to their indices. + final DrillBuf drillBuf = rootAllocator.buffer(256, 256 + 256); + assertEquals(256, drillBuf.capacity()); + assertEquals(256 + 256, drillBuf.maxCapacity()); + assertEquals(0, drillBuf.readerIndex()); + assertEquals(0, drillBuf.readableBytes()); + assertEquals(0, drillBuf.writerIndex()); + assertEquals(256, drillBuf.writableBytes()); + + final DrillBuf slice3 = (DrillBuf) drillBuf.slice(); + assertEquals(0, slice3.readerIndex()); + assertEquals(0, slice3.readableBytes()); + assertEquals(0, slice3.writerIndex()); +// assertEquals(256, slice3.capacity()); +// assertEquals(256, slice3.writableBytes()); + + for(int i = 0; i < 256; ++i) { + drillBuf.writeByte(i); + } + assertEquals(0, drillBuf.readerIndex()); + assertEquals(256, drillBuf.readableBytes()); + assertEquals(256, drillBuf.writerIndex()); + assertEquals(0, drillBuf.writableBytes()); + + final DrillBuf slice1 = (DrillBuf) drillBuf.slice(); + assertEquals(0, slice1.readerIndex()); + assertEquals(256, slice1.readableBytes()); + for(int i = 0; i < 10; ++i) { + assertEquals(i, slice1.readByte()); + } + assertEquals(256 - 10, slice1.readableBytes()); + for(int i = 0; i < 256; ++i) { + assertEquals((byte) i, slice1.getByte(i)); + } + + final DrillBuf slice2 = (DrillBuf) drillBuf.slice(25, 25); + assertEquals(0, slice2.readerIndex()); + assertEquals(25, slice2.readableBytes()); + for(int i = 25; i < 50; ++i) { + assertEquals(i, slice2.readByte()); + } + +/* + for(int i = 256; i > 0; --i) { + slice3.writeByte(i - 1); + } + for(int i = 0; i < 256; ++i) { + assertEquals(255 - i, slice1.getByte(i)); + } +*/ + + drillBuf.release(); // all the derived buffers share this fate + } + } + + @Test + public void testAllocator_slicesOfSlices() throws Exception { +// final AllocatorOwner allocatorOwner = new NamedOwner("slicesOfSlices"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + // Populate a buffer with byte values corresponding to their indices. + final DrillBuf drillBuf = rootAllocator.buffer(256, 256 + 256); + for(int i = 0; i < 256; ++i) { + drillBuf.writeByte(i); + } + + // Slice it up. + final DrillBuf slice0 = drillBuf.slice(0, drillBuf.capacity()); + for(int i = 0; i < 256; ++i) { + assertEquals((byte) i, drillBuf.getByte(i)); + } + + final DrillBuf slice10 = slice0.slice(10, drillBuf.capacity() - 10); + for(int i = 10; i < 256; ++i) { + assertEquals((byte) i, slice10.getByte(i - 10)); + } + + final DrillBuf slice20 = slice10.slice(10, drillBuf.capacity() - 20); + for(int i = 20; i < 256; ++i) { + assertEquals((byte) i, slice20.getByte(i - 20)); + } + + final DrillBuf slice30 = slice20.slice(10, drillBuf.capacity() - 30); + for(int i = 30; i < 256; ++i) { + assertEquals((byte) i, slice30.getByte(i - 30)); + } + + drillBuf.release(); + } + } + + @Test + public void testAllocator_transferSliced() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("transferSliced"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + final BufferAllocator childAllocator1 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + final BufferAllocator childAllocator2 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + + final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 8); + final DrillBuf drillBuf2 = childAllocator2.buffer(MAX_ALLOCATION / 8); + + final DrillBuf drillBuf1s = drillBuf1.slice(0, drillBuf1.capacity() / 2); + final DrillBuf drillBuf2s = drillBuf2.slice(0, drillBuf2.capacity() / 2); + + rootAllocator.verify(); + + childAllocator1.takeOwnership(drillBuf2s); + rootAllocator.verify(); + childAllocator2.takeOwnership(drillBuf1s); + rootAllocator.verify(); + + drillBuf1s.release(); // releases drillBuf1 + drillBuf2s.release(); // releases drillBuf2 + + childAllocator1.close(); + childAllocator2.close(); + } + } + + @Test + public void testAllocator_shareSliced() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("transferSliced"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + final BufferAllocator childAllocator1 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + final BufferAllocator childAllocator2 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + + final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 8); + final DrillBuf drillBuf2 = childAllocator2.buffer(MAX_ALLOCATION / 8); + + final DrillBuf drillBuf1s = drillBuf1.slice(0, drillBuf1.capacity() / 2); + final DrillBuf drillBuf2s = drillBuf2.slice(0, drillBuf2.capacity() / 2); + + rootAllocator.verify(); + + final Pointer pDrillBuf = new Pointer<>(); + childAllocator1.shareOwnership(drillBuf2s, pDrillBuf); + final DrillBuf drillBuf2s1 = pDrillBuf.value; + childAllocator2.shareOwnership(drillBuf1s, pDrillBuf); + final DrillBuf drillBuf1s2 = pDrillBuf.value; + rootAllocator.verify(); + + drillBuf1s.release(); // releases drillBuf1 + drillBuf2s.release(); // releases drillBuf2 + rootAllocator.verify(); + + drillBuf2s1.release(); // releases the shared drillBuf2 slice + drillBuf1s2.release(); // releases the shared drillBuf1 slice + + childAllocator1.close(); + childAllocator2.close(); + } + } + + @Test + public void testAllocator_transferShared() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("transferShared"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + final BufferAllocator childAllocator1 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + final BufferAllocator childAllocator2 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + final BufferAllocator childAllocator3 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + + final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 8); + + final Pointer pDrillBuf = new Pointer<>(); + boolean allocationFit; + + allocationFit = childAllocator2.shareOwnership(drillBuf1, pDrillBuf); + assertTrue(allocationFit); + rootAllocator.verify(); + final DrillBuf drillBuf2 = pDrillBuf.value; + assertNotNull(drillBuf2); + assertNotEquals(drillBuf2, drillBuf1); + + allocationFit = childAllocator3.takeOwnership(drillBuf1); + assertTrue(allocationFit); + rootAllocator.verify(); + + // Since childAllocator3 now has childAllocator1's buffer, 1, can close + childAllocator1.close(); + rootAllocator.verify(); + + drillBuf2.release(); + childAllocator2.close(); + rootAllocator.verify(); + + final BufferAllocator childAllocator4 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0); + allocationFit = childAllocator4.takeOwnership(drillBuf1); + assertTrue(allocationFit); + rootAllocator.verify(); + + childAllocator3.close(); + rootAllocator.verify(); + + drillBuf1.release(); + childAllocator4.close(); + rootAllocator.verify(); + } + } + + @Test + public void testAllocator_unclaimedReservation() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("unclaimedReservation"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + try(final BufferAllocator childAllocator1 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + try(final AllocationReservation reservation = childAllocator1.newReservation()) { + assertTrue(reservation.add(64)); + } + rootAllocator.verify(); + } + } + } + + @Test + public void testAllocator_claimedReservation() throws Exception { + final AllocatorOwner allocatorOwner = new NamedOwner("claimedReservation"); + try(final RootAllocator rootAllocator = + new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) { + try(final BufferAllocator childAllocator1 = + rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) { + try(final AllocationReservation reservation = childAllocator1.newReservation()) { + assertTrue(reservation.add(32)); + assertTrue(reservation.add(32)); + + final DrillBuf drillBuf = reservation.buffer(); + assertEquals(64, drillBuf.capacity()); + rootAllocator.verify(); + + drillBuf.release(); + rootAllocator.verify(); + } + rootAllocator.verify(); + } + } + } +} diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestEndianess.java b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestEndianess.java index 48ddada9bb7..89ecde3a78d 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestEndianess.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestEndianess.java @@ -20,22 +20,21 @@ import static org.junit.Assert.assertEquals; import io.netty.buffer.ByteBuf; +import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.ExecTest; import org.junit.Test; - - -public class TestEndianess extends ExecTest{ - +public class TestEndianess extends ExecTest { @Test - public void testLittleEndian(){ - TopLevelAllocator a = new TopLevelAllocator(Long.MAX_VALUE); - ByteBuf b = a.buffer(4); - b.setInt(0, 35); - assertEquals((int) b.getByte(0), 35); - assertEquals((int) b.getByte(1), 0); - assertEquals((int) b.getByte(2), 0); - assertEquals((int) b.getByte(3), 0); + public void testLittleEndian() throws Exception { + try(final RootAllocator a = new RootAllocator(DrillConfig.create())) { + final ByteBuf b = a.buffer(4); + b.setInt(0, 35); + assertEquals((int) b.getByte(0), 35); + assertEquals((int) b.getByte(1), 0); + assertEquals((int) b.getByte(2), 0); + assertEquals((int) b.getByte(3), 0); + b.release(); + } } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java index 42d2193f5a0..111ceb32ee8 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java @@ -21,6 +21,7 @@ import java.util.List; import org.apache.drill.common.DeferredException; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.ops.FragmentContext.ExecutorState; @@ -134,5 +135,4 @@ public int getRecordCount() { public RecordBatch getIncoming() { return incoming; } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java index ffa87650575..5d63a38a547 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java @@ -40,7 +40,7 @@ import org.apache.drill.exec.expr.holders.VarBinaryHolder; import org.apache.drill.exec.expr.holders.VarCharHolder; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -69,18 +69,14 @@ import com.google.common.io.Files; public class TestCastFunctions extends PopUnitTestBase{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFunctions.class); - - DrillConfig c = DrillConfig.create(); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFunctions.class); + private final DrillConfig c = DrillConfig.create(); @Test - // cast to bigint. public void testCastBigInt(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ - - final BufferAllocator allocator = new TopLevelAllocator(); - + final BufferAllocator allocator = new RootAllocator(c); new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); bitContext.getAllocator(); result = allocator; @@ -123,12 +119,10 @@ public void testCastBigInt(@Injectable final DrillbitContext bitContext, } @Test - //cast to int public void testCastInt(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ - final BufferAllocator allocator = new TopLevelAllocator(); - + final BufferAllocator allocator = new RootAllocator(c); new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); bitContext.getAllocator(); result = allocator; @@ -171,10 +165,9 @@ public void testCastInt(@Injectable final DrillbitContext bitContext, } @Test - //cast to float4 public void testCastFloat4(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ - final BufferAllocator allocator = new TopLevelAllocator(); + final BufferAllocator allocator = new RootAllocator(c); new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); bitContext.getAllocator(); result = allocator; @@ -217,11 +210,10 @@ public void testCastFloat4(@Injectable final DrillbitContext bitContext, } @Test - //cast to float8 public void testCastFloat8(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ - final BufferAllocator allocator = new TopLevelAllocator(); + final BufferAllocator allocator = new RootAllocator(c); new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); bitContext.getAllocator(); result = allocator; @@ -264,11 +256,10 @@ public void testCastFloat8(@Injectable final DrillbitContext bitContext, } @Test - //cast to varchar(length) public void testCastVarChar(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ - final BufferAllocator allocator = new TopLevelAllocator(); + final BufferAllocator allocator = new RootAllocator(c); new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); bitContext.getAllocator(); result = allocator; @@ -311,11 +302,10 @@ public void testCastVarChar(@Injectable final DrillbitContext bitContext, } @Test - //cast to varbinary(length) public void testCastVarBinary(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ - final BufferAllocator allocator = new TopLevelAllocator(); + final BufferAllocator allocator = new RootAllocator(c); new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); @@ -361,8 +351,7 @@ public void testCastVarBinary(@Injectable final DrillbitContext bitContext, //nested: cast is nested in another cast, or another function. public void testCastNested(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ - - final BufferAllocator allocator = new TopLevelAllocator(); + final BufferAllocator allocator = new RootAllocator(c); new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); bitContext.getAllocator(); result = allocator; @@ -409,7 +398,7 @@ public void testCastNested(@Injectable final DrillbitContext bitContext, public void testCastNumException(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ - final BufferAllocator allocator = new TopLevelAllocator(); + final BufferAllocator allocator = new RootAllocator(c); new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); @@ -442,7 +431,7 @@ public void testCastNumException(@Injectable final DrillbitContext bitContext, } @Test - public void testCastFromNullablCol() throws Throwable { + public void testCastFromNullableCol() throws Throwable { RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); try(Drillbit bit = new Drillbit(CONFIG, serviceSet); @@ -486,15 +475,15 @@ public void testCastFromNullablCol() throws Throwable { private Object[][] getRunResult(VectorAccessible va) { int size = 0; - for (VectorWrapper v : va) { + for (@SuppressWarnings("unused") final VectorWrapper v : va) { size++; } - Object[][] res = new Object [va.getRecordCount()][size]; + final Object[][] res = new Object[va.getRecordCount()][size]; for (int j = 0; j < va.getRecordCount(); j++) { int i = 0; - for (VectorWrapper v : va) { - Object o = v.getValueVector().getAccessor().getObject(j); + for (final VectorWrapper v : va) { + final Object o = v.getValueVector().getAccessor().getObject(j); if (o instanceof byte[]) { res[j][i++] = new String((byte[]) o); } else { @@ -504,5 +493,4 @@ private Object[][] getRunResult(VectorAccessible va) { } return res; } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java index c69c6f59c41..8ff61fe67aa 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java @@ -26,7 +26,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -54,9 +54,9 @@ public void runTest(@Injectable final DrillbitContext bitContext, new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); }}; @@ -154,5 +154,4 @@ public void testBigIntNullable(@Injectable final DrillbitContext bitContext, runTest(bitContext, connection, "bigIntNullableColumn >= bigIntNullableColumn", 50); runTest(bitContext, connection, "bigIntNullableColumn <= bigIntNullableColumn", 50); } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java index 03c6f412b3d..c7edbbfb236 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java @@ -26,7 +26,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -43,9 +43,9 @@ import com.google.common.io.Resources; public class TestImplicitCastFunctions extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestImplicitCastFunctions.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestImplicitCastFunctions.class); - DrillConfig c = DrillConfig.create(); + final DrillConfig c = DrillConfig.create(); PhysicalPlanReader reader; FunctionImplementationRegistry registry; FragmentContext context; @@ -69,9 +69,9 @@ public void runTest(@Injectable final DrillbitContext bitContext, new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); }}; @@ -170,5 +170,4 @@ public void testImplicitCastWithNullExpression(@Injectable final DrillbitContext runTest(bitContext, connection, expected, "functions/cast/testICastNullExp.json"); } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java index dc37071fa47..b8143c27301 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java @@ -30,7 +30,7 @@ import org.apache.drill.exec.coord.ClusterCoordinator; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; import org.apache.drill.exec.expr.holders.VarBinaryHolder; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.opt.BasicOptimizer; @@ -66,8 +66,8 @@ @Ignore public class TestOptiqPlans extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOptiqPlans.class); - DrillConfig c = DrillConfig.create(); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOptiqPlans.class); + final DrillConfig c = DrillConfig.create(); @Test public void orderBy(@Injectable final BootStrapContext ctxt, @Injectable UserClientConnection connection, @@ -96,13 +96,13 @@ private SimpleRootExec doLogicalTest(final BootStrapContext context, UserClientC { context.getMetrics(); result = new MetricRegistry(); - context.getAllocator(); - result = new TopLevelAllocator(); context.getConfig(); result = c; + context.getAllocator(); + result = new RootAllocator(c); } }; - RemoteServiceSet lss = RemoteServiceSet.getLocalServiceSet(); + DrillbitContext bitContext = new DrillbitContext(DrillbitEndpoint.getDefaultInstance(), context, coord, controller, com, workBus, new LocalPStoreProvider(DrillConfig.create()), null); QueryContext qc = new QueryContext(UserSession.Builder.newBuilder().setSupportComplexTypes(true).build(), @@ -139,7 +139,6 @@ public void testFilterPlan() throws Exception { System.out.println(vw.getValueVector().getField().toExpr()); ValueVector vv = vw.getValueVector(); for (int i = 0; i < vv.getAccessor().getValueCount(); i++) { - Object o = vv.getAccessor().getObject(i); System.out.println(vv.getAccessor().getObject(i)); } } @@ -169,7 +168,6 @@ public void testJoinPlan() throws Exception { System.out.println(vw.getValueVector().getField().toExpr()); ValueVector vv = vw.getValueVector(); for (int i = 0; i < vv.getAccessor().getValueCount(); i++) { - Object o = vv.getAccessor().getObject(i); System.out.println(vv.getAccessor().getObject(i)); } } @@ -199,7 +197,6 @@ public void testFilterString() throws Exception { System.out.println(vw.getValueVector().getField().toExpr()); ValueVector vv = vw.getValueVector(); for (int i = 0; i < vv.getAccessor().getValueCount(); i++) { - Object o = vv.getAccessor().getObject(i); if (vv instanceof VarBinaryVector) { VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor(); VarBinaryHolder vbh = new VarBinaryHolder(); @@ -239,7 +236,6 @@ public void testLogicalJsonScan() throws Exception { System.out.println(vw.getValueVector().getField().toExpr()); ValueVector vv = vw.getValueVector(); for (int i = 0; i < vv.getAccessor().getValueCount(); i++) { - Object o = vv.getAccessor().getObject(i); if (vv instanceof VarBinaryVector) { VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor(); VarBinaryHolder vbh = new VarBinaryHolder(); @@ -279,7 +275,6 @@ public void testOrderVarbinary() throws Exception { System.out.println(vw.getValueVector().getField().toExpr()); ValueVector vv = vw.getValueVector(); for (int i = 0; i < vv.getAccessor().getValueCount(); i++) { - Object o = vv.getAccessor().getObject(i); if (vv instanceof VarBinaryVector) { VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor(); VarBinaryHolder vbh = new VarBinaryHolder(); @@ -307,7 +302,7 @@ private SimpleRootExec doPhysicalTest(final DrillbitContext bitContext, UserClie bitContext.getMetrics(); result = new MetricRegistry(); bitContext.getAllocator(); - result = new TopLevelAllocator(); + result = new RootAllocator(c); bitContext.getConfig(); result = c; } @@ -324,5 +319,4 @@ private SimpleRootExec doPhysicalTest(final DrillbitContext bitContext, UserClie .iterator().next())); return exec; } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java index 09ba1a5e91b..7231520cc57 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java @@ -21,12 +21,12 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import java.nio.charset.Charset; import java.util.List; import org.apache.drill.common.util.FileUtils; import org.apache.drill.exec.client.DrillClient; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.BufferAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.pop.PopUnitTestBase; import org.apache.drill.exec.proto.UserBitShared.QueryType; import org.apache.drill.exec.record.RecordBatchLoader; @@ -41,9 +41,7 @@ import com.google.common.io.Files; public class TestSimpleFragmentRun extends PopUnitTestBase { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFragmentRun.class); - - private static final Charset UTF_8 = Charset.forName("UTF-8"); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFragmentRun.class); @Test public void runNoExchangeFragment() throws Exception { @@ -121,7 +119,8 @@ public void runJSONScanPopFragment() throws Exception { ); // look at records - RecordBatchLoader batchLoader = new RecordBatchLoader(new TopLevelAllocator(CONFIG)); + final BufferAllocator allocator = new RootAllocator(CONFIG); + RecordBatchLoader batchLoader = new RecordBatchLoader(allocator); int recordCount = 0; //int expectedBatchCount = 2; @@ -139,17 +138,11 @@ public void runJSONScanPopFragment() throws Exception { } assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData())); - boolean firstColumn = true; // print headers. System.out.println("\n\n========NEW SCHEMA=========\n\n"); - for (VectorWrapper v : batchLoader) { - - if (firstColumn) { - firstColumn = false; - } else { - System.out.print("\t"); - } + for (final VectorWrapper v : batchLoader) { + System.out.print("\t"); // always use for r[N] pseudo-column used for record number below System.out.print(v.getField().toExpr()); System.out.print("["); System.out.print(v.getField().getType().getMinorType()); @@ -159,20 +152,22 @@ public void runJSONScanPopFragment() throws Exception { System.out.println(); - for (int r = 0; r < batchLoader.getRecordCount(); r++) { - boolean first = true; + final int loaderRecordCount = batchLoader.getRecordCount(); + for (int r = 0; r < loaderRecordCount; r++) { + boolean firstColumn = true; recordCount++; - for (VectorWrapper v : batchLoader) { - if (first) { - first = false; + for (final VectorWrapper v : batchLoader) { + if (firstColumn) { + firstColumn = false; + System.out.print("r[" + r + "]\t"); } else { System.out.print("\t"); } - ValueVector.Accessor accessor = v.getValueVector().getAccessor(); + final ValueVector.Accessor accessor = v.getValueVector().getAccessor(); System.out.print(accessor.getObject(r)); } - if (!first) { + if (!firstColumn) { System.out.println(); } } @@ -183,5 +178,4 @@ public void runJSONScanPopFragment() throws Exception { assertEquals(2, recordCount); } } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java index d551319ea98..a2a0a692547 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java @@ -20,7 +20,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -44,7 +43,7 @@ import org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers; import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder; import org.apache.drill.exec.expr.holders.NullableVarCharHolder; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -62,15 +61,14 @@ import com.codahale.metrics.MetricRegistry; import com.google.common.base.Charsets; import com.google.common.io.Files; -import com.sun.codemodel.JClassAlreadyExistsException; public class TestSimpleFunctions extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFunctions.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFunctions.class); - DrillConfig c = DrillConfig.create(); + final DrillConfig c = DrillConfig.create(); @Test - public void testHashFunctionResolution(@Injectable DrillConfig config) throws JClassAlreadyExistsException, IOException { + public void testHashFunctionResolution(@Injectable DrillConfig config) { FunctionImplementationRegistry registry = new FunctionImplementationRegistry(config); // test required vs nullable Int input resolveHash(config, @@ -135,10 +133,9 @@ public void testHashFunctionResolution(@Injectable DrillConfig config) throws JC public void resolveHash(DrillConfig config, LogicalExpression arg, TypeProtos.MajorType expectedArg, TypeProtos.MajorType expectedOut, TypeProtos.DataMode expectedBestInputMode, - FunctionImplementationRegistry registry) throws JClassAlreadyExistsException, IOException { + FunctionImplementationRegistry registry) { List args = new ArrayList<>(); args.add(arg); - String[] registeredNames = { "hash" }; FunctionCall call = new FunctionCall( "hash", args, @@ -155,7 +152,7 @@ public void testSubstring(@Injectable final DrillbitContext bitContext, new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -197,7 +194,7 @@ public void testSubstringNegative(@Injectable final DrillbitContext bitContext, new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -240,7 +237,7 @@ public void testByteSubstring(@Injectable final DrillbitContext bitContext, new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -273,7 +270,5 @@ public void testByteSubstring(@Injectable final DrillbitContext bitContext, throw context.getFailureCause(); } assertTrue(!context.isFailed()); - } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestStringFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestStringFunctions.java index d72c1e1e321..7e8935eb2c3 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestStringFunctions.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestStringFunctions.java @@ -26,7 +26,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -44,7 +44,7 @@ import com.google.common.io.Resources; public class TestStringFunctions extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestStringFunctions.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestStringFunctions.class); DrillConfig c = DrillConfig.create(); PhysicalPlanReader reader; @@ -53,7 +53,7 @@ public class TestStringFunctions extends ExecTest { public Object[] getRunResult(SimpleRootExec exec) { int size = 0; - for (ValueVector v : exec) { + for (@SuppressWarnings("unused") ValueVector v : exec) { size++; } @@ -73,7 +73,7 @@ public void runTest(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection, Object[] expectedResults, String planPath) throws Throwable { new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java index 9c24f79841f..b30b857cb83 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java @@ -33,24 +33,24 @@ import com.google.common.io.Files; public class TestUnionExchange extends PopUnitTestBase { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestUnionExchange.class); + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestUnionExchange.class); @Test public void twoBitTwoExchangeTwoEntryRun() throws Exception { - RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); + final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); - try (Drillbit bit1 = new Drillbit(CONFIG, serviceSet); - Drillbit bit2 = new Drillbit(CONFIG, serviceSet); - DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) { + try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet); + final Drillbit bit2 = new Drillbit(CONFIG, serviceSet); + final DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) { bit1.run(); bit2.run(); client.connect(); - List results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL, + final List results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/sender/union_exchange.json"), Charsets.UTF_8)); int count = 0; - for (QueryDataBatch b : results) { + for (final QueryDataBatch b : results) { if (b.getHeader().getRowCount() != 0) { count += b.getHeader().getRowCount(); } @@ -59,5 +59,4 @@ public void twoBitTwoExchangeTwoEntryRun() throws Exception { assertEquals(150, count); } } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java index d2616a8baf5..d1987079b3a 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java @@ -28,7 +28,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -50,13 +50,13 @@ import com.google.common.io.Files; public class TestAgg extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestAgg.class); - DrillConfig c = DrillConfig.create(); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestAgg.class); + final DrillConfig c = DrillConfig.create(); private SimpleRootExec doTest(final DrillbitContext bitContext, UserClientConnection connection, String file) throws Exception{ new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -127,7 +127,5 @@ public void twoKeyAgg(@Injectable final DrillbitContext bitContext, @Injectable throw exec.getContext().getFailureCause(); } assertTrue(!exec.getContext().isFailed()); - } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/TestHashTable.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/TestHashTable.java index b02249d3147..360b64d759e 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/TestHashTable.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/TestHashTable.java @@ -24,7 +24,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -42,21 +42,19 @@ import com.google.common.io.Files; public class TestHashTable extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestHashTable.class); - DrillConfig c = DrillConfig.create(); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestHashTable.class); + final DrillConfig c = DrillConfig.create(); @SuppressWarnings("deprecation") private SimpleRootExec doTest(final DrillbitContext bitContext, UserClientConnection connection, String plan_path) throws Exception{ new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); }}; - - PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance()); PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(plan_path), Charsets.UTF_8)); FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c); diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java index a069078ef7d..53403331929 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java @@ -27,7 +27,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -47,24 +47,19 @@ import com.google.common.io.Files; public class TestSimpleFilter extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFilter.class); - DrillConfig c = DrillConfig.create(); - +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFilter.class); + private final DrillConfig c = DrillConfig.create(); @Test public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{ -// System.out.println(System.getProperty("java.class.path")); - - new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); }}; - PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance()); PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/filter/test1.json"), Charsets.UTF_8)); FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c); @@ -80,7 +75,6 @@ public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable throw context.getFailureCause(); } assertTrue(!context.isFailed()); - } @Test @@ -88,7 +82,7 @@ public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable public void testSV4Filter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{ new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -113,7 +107,5 @@ public void testSV4Filter(@Injectable final DrillbitContext bitContext, @Injecta throw context.getFailureCause(); } assertTrue(!context.isFailed()); - } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java index 6c067febea3..2481a81f9d2 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java @@ -32,7 +32,7 @@ import org.apache.drill.exec.client.DrillClient; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -61,13 +61,12 @@ import com.google.common.base.Charsets; import com.google.common.io.Files; - public class TestHashJoin extends PopUnitTestBase { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMergeJoin.class); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMergeJoin.class); @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(100000); - DrillConfig c = DrillConfig.create(); + private final DrillConfig c = DrillConfig.create(); private void testHJMockScanCommon(final DrillbitContext bitContext, UserServer.UserClientConnection connection, String physicalPlan, int expectedRows) throws Throwable { final LocalPStoreProvider provider = new LocalPStoreProvider(c); @@ -76,7 +75,7 @@ private void testHJMockScanCommon(final DrillbitContext bitContext, UserServer.U opt.init(); new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getOptionManager(); result = opt; @@ -105,21 +104,18 @@ private void testHJMockScanCommon(final DrillbitContext bitContext, UserServer.U @Test public void multiBatchEqualityJoin(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable { - testHJMockScanCommon(bitContext, connection, "/join/hash_join_multi_batch.json", 200000); } @Test public void multiBatchRightOuterJoin(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable { - testHJMockScanCommon(bitContext, connection, "/join/hj_right_outer_multi_batch.json", 100000); } @Test public void multiBatchLeftOuterJoin(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable { - testHJMockScanCommon(bitContext, connection, "/join/hj_left_outer_multi_batch.json", 100000); } @@ -151,7 +147,6 @@ public void simpleEqualityJoin() throws Throwable { // Check the output of decimal9 ValueVector.Accessor intAccessor1 = itr.next().getValueVector().getAccessor(); - for (int i = 0; i < intAccessor1.getValueCount(); i++) { assertEquals(intAccessor1.getObject(i), colA[i]); } @@ -167,7 +162,6 @@ public void simpleEqualityJoin() throws Throwable { @Test public void hjWithExchange(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable { - // Function tests with hash join with exchanges try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); Drillbit bit = new Drillbit(CONFIG, serviceSet); @@ -224,7 +218,6 @@ public void multipleConditionJoin(@Injectable final DrillbitContext bitContext, ValueVector.Accessor intAccessor1 = itr.next().getValueVector().getAccessor(); ValueVector.Accessor intAccessor2 = itr.next().getValueVector().getAccessor(); - for (int i = 0; i < intAccessor1.getValueCount(); i++) { assertEquals(intAccessor1.getObject(i), colA[i]); assertEquals(intAccessor2.getObject(i), colC[i]); @@ -238,11 +231,9 @@ public void multipleConditionJoin(@Injectable final DrillbitContext bitContext, } } - @Test public void hjWithExchange1(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable { - // Another test for hash join with exchanges try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); Drillbit bit = new Drillbit(CONFIG, serviceSet); @@ -267,14 +258,12 @@ public void hjWithExchange1(@Injectable final DrillbitContext bitContext, } } - @Test public void testHashJoinExprInCondition() throws Exception { - RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); - - try (Drillbit bit1 = new Drillbit(CONFIG, serviceSet); - DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) { + final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); + try (final Drillbit bit1 = new Drillbit(CONFIG, serviceSet); + final DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) { bit1.run(); client.connect(); List results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL, @@ -289,5 +278,4 @@ public void testHashJoinExprInCondition() throws Exception { assertEquals(10, count); } } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java index 18555c753cb..3c640eff247 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java @@ -31,7 +31,7 @@ import org.apache.drill.exec.client.DrillClient; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -60,9 +60,8 @@ public class TestMergeJoin extends PopUnitTestBase { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMergeJoin.class); - - DrillConfig c = DrillConfig.create(); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMergeJoin.class); + private final DrillConfig c = DrillConfig.create(); @Test @Ignore // this doesn't have a sort. it also causes an infinite loop. these may or may not be related. @@ -71,7 +70,7 @@ public void simpleEqualityJoin(@Injectable final DrillbitContext bitContext, new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -126,7 +125,7 @@ public void orderedEqualityLeftJoin(@Injectable final DrillbitContext bitContext new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getConfig(); result = c; bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -184,7 +183,7 @@ public void orderedEqualityInnerJoin(@Injectable final DrillbitContext bitContex new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getConfig(); result = c; bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -242,7 +241,7 @@ public void orderedEqualityMultiBatchJoin(@Injectable final DrillbitContext bitC new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getConfig(); result = c; bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -296,7 +295,7 @@ public void orderedEqualityMultiBatchJoin(@Injectable final DrillbitContext bitC public void testJoinBatchSize(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{ new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator();; + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getConfig(); result = c; bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -412,5 +411,4 @@ public void testMergeJoinExprInCondition() throws Exception { assertEquals(10, count); } } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java index 7cdb41a31cf..9c83671601b 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java @@ -27,7 +27,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -48,13 +48,13 @@ import com.google.common.io.Files; public class TestSimpleLimit extends ExecTest { - DrillConfig c = DrillConfig.create(); + private final DrillConfig c = DrillConfig.create(); @Test public void testLimit(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -67,7 +67,7 @@ public void testLimit(@Injectable final DrillbitContext bitContext, @Injectable public void testLimitNoEnd(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -85,7 +85,7 @@ public void testLimitNoEnd(@Injectable final DrillbitContext bitContext, @Inject public void testLimitAcrossBatches(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -97,8 +97,6 @@ public void testLimitAcrossBatches(@Injectable final DrillbitContext bitContext, long expectedSum = (end - start) * (end + start - 1) / 2; //Formula for sum of series verifySum(bitContext, connection, "test4.json", 70000, expectedSum); - - } private void verifyLimitCount(DrillbitContext bitContext, UserServer.UserClientConnection connection, String testPlan, int expectedCount) throws Throwable { diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java index f57d7a948cd..80bf17ebc85 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java @@ -26,11 +26,14 @@ import org.apache.drill.common.util.FileUtils; import org.apache.drill.exec.client.DrillClient; import org.apache.drill.exec.pop.PopUnitTestBase; +import org.apache.drill.exec.proto.UserBitShared.QueryData; +import org.apache.drill.exec.record.MaterializedField; import org.apache.drill.exec.record.RecordBatchLoader; import org.apache.drill.exec.record.VectorWrapper; import org.apache.drill.exec.rpc.user.QueryDataBatch; import org.apache.drill.exec.server.Drillbit; import org.apache.drill.exec.server.RemoteServiceSet; +import org.apache.drill.exec.vector.ValueVector; import org.junit.Test; import com.google.common.base.Charsets; @@ -57,26 +60,10 @@ public void twoBitTwoExchange() throws Exception { final RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator()); // print the results for (final QueryDataBatch b : results) { - count += b.getHeader().getRowCount(); - for (int valueIdx = 0; valueIdx < b.getHeader().getRowCount(); valueIdx++) { - final List row = Lists.newArrayList(); - batchLoader.load(b.getHeader().getDef(), b.getData()); - for (final VectorWrapper vw : batchLoader) { - row.add(vw.getValueVector().getField().toExpr() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx)); - } - for (final Object cell : row) { - if (cell == null) { -// System.out.print(" "); - continue; - } - final int len = cell.toString().length(); -// System.out.print(cell + " "); - for (int i = 0; i < (30 - len); ++i) { -// System.out.print(" "); - } - } -// System.out.println(); - } + final QueryData queryData = b.getHeader(); + final int rowCount = queryData.getRowCount(); + count += rowCount; + batchLoader.load(queryData.getDef(), b.getData()); // loaded but not used, just to test b.release(); batchLoader.clear(); } @@ -104,31 +91,25 @@ public void testMultipleProvidersMixedSizes() throws Exception { // print the results Long lastBlueValue = null; for (final QueryDataBatch b : results) { - count += b.getHeader().getRowCount(); - for (int valueIdx = 0; valueIdx < b.getHeader().getRowCount(); valueIdx++) { - final List row = Lists.newArrayList(); - batchLoader.load(b.getHeader().getDef(), b.getData()); - for (final VectorWrapper vw : batchLoader) { - row.add(vw.getValueVector().getField().toExpr() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx)); - if (vw.getValueVector().getField().getAsSchemaPath().getRootSegment().getPath().equals("blue")) { + final QueryData queryData = b.getHeader(); + final int batchRowCount = queryData.getRowCount(); + count += batchRowCount; + batchLoader.load(queryData.getDef(), b.getData()); + for (final VectorWrapper vw : batchLoader) { + final ValueVector vv = vw.getValueVector(); + final ValueVector.Accessor va = vv.getAccessor(); + final MaterializedField materializedField = vv.getField(); + final int numValues = va.getValueCount(); + for(int valueIdx = 0; valueIdx < numValues; ++valueIdx) { + if (materializedField.getAsSchemaPath().getRootSegment().getPath().equals("blue")) { + final long longValue = ((Long) va.getObject(valueIdx)).longValue(); // check that order is ascending - if (((Long)vw.getValueVector().getAccessor().getObject(valueIdx)).longValue() == 0) { - continue; // ignore initial 0's from sort - } if (lastBlueValue != null) { - assertTrue(((Long)vw.getValueVector().getAccessor().getObject(valueIdx)).longValue() >= ((Long)lastBlueValue).longValue()); + assertTrue(longValue >= lastBlueValue); } - lastBlueValue = (Long)vw.getValueVector().getAccessor().getObject(valueIdx); + lastBlueValue = longValue; } } - for (final Object cell : row) { - int len = cell.toString().length(); -// System.out.print(cell + " "); - for (int i = 0; i < (30 - len); ++i) { -// System.out.print(" "); - } - } -// System.out.println(); } b.release(); batchLoader.clear(); @@ -156,26 +137,9 @@ public void handleEmptyBatch() throws Exception { final RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator()); // print the results for (final QueryDataBatch b : results) { - count += b.getHeader().getRowCount(); - for (int valueIdx = 0; valueIdx < b.getHeader().getRowCount(); valueIdx++) { - final List row = Lists.newArrayList(); - batchLoader.load(b.getHeader().getDef(), b.getData()); - for (final VectorWrapper vw : batchLoader) { - row.add(vw.getValueVector().getField().toExpr() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx)); - } - for (final Object cell : row) { - if (cell == null) { -// System.out.print(" "); - continue; - } - int len = cell.toString().length(); -// System.out.print(cell + " "); - for (int i = 0; i < (30 - len); ++i) { -// System.out.print(" "); - } - } -// System.out.println(); - } + final QueryData queryData = b.getHeader(); + batchLoader.load(queryData.getDef(), b.getData()); // loaded but not used, for testing + count += queryData.getRowCount(); b.release(); batchLoader.clear(); } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java index 43c430a8805..25631dd9c14 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java @@ -29,7 +29,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -50,15 +50,15 @@ import com.google.common.io.Files; public class TestSimpleProjection extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleProjection.class); - DrillConfig c = DrillConfig.create(); +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleProjection.class); + private final DrillConfig c = DrillConfig.create(); @Test public void project(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{ new NonStrictExpectations() {{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -92,5 +92,4 @@ public void project(@Injectable final DrillbitContext bitContext, @Injectable Us } assertTrue(!context.isFailed()); } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java index d51a017d7e3..4b9a053f2e0 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java @@ -29,7 +29,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -59,7 +59,7 @@ public class TestSimpleSort extends ExecTest { public void sortOneKeyAscending(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{ new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -103,7 +103,7 @@ public void sortOneKeyAscending(@Injectable final DrillbitContext bitContext, @I public void sortTwoKeysOneAscendingOneDescending(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{ new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(bitContext.getConfig()); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -139,7 +139,6 @@ public void sortTwoKeysOneAscendingOneDescending(@Injectable final DrillbitConte } assertTrue(previousLong >= a2.get(i)); - //System.out.println(previousInt + "\t" + a2.get(i)); } } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java index b82846e6d4a..1381b685c84 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java @@ -26,7 +26,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -54,17 +54,15 @@ * incoming container of the trace operator. */ public class TestTraceMultiRecordBatch extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTraceOutputDump.class); - DrillConfig c = DrillConfig.create(); - +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTraceOutputDump.class); + private final DrillConfig c = DrillConfig.create(); @Test public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable { - new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getConfig(); result = c; bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -89,4 +87,4 @@ public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable } assertTrue(!context.isFailed()); } -} \ No newline at end of file +} diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java index 1cb72ffa65e..5009c81617b 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java @@ -17,6 +17,8 @@ */ package org.apache.drill.exec.physical.impl.trace; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import mockit.Injectable; import mockit.NonStrictExpectations; @@ -28,7 +30,7 @@ import org.apache.drill.exec.cache.VectorAccessibleSerializable; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -68,17 +70,15 @@ * known value. */ public class TestTraceOutputDump extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTraceOutputDump.class); - DrillConfig c = DrillConfig.create(); - +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTraceOutputDump.class); + private final DrillConfig c = DrillConfig.create(); @Test public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable { - new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getConfig(); result = c; bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); @@ -98,7 +98,7 @@ public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable if(context.getFailureCause() != null){ throw context.getFailureCause(); } - assertTrue(!context.isFailed()); + assertFalse(context.isFailed()); FragmentHandle handle = context.getHandle(); @@ -116,10 +116,10 @@ public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable System.out.println("File Name: " + filename); - Configuration conf = new Configuration(); + Configuration conf = new Configuration(); conf.set(FileSystem.FS_DEFAULT_NAME_KEY, c.getString(ExecConstants.TRACE_DUMP_FILESYSTEM)); - FileSystem fs = FileSystem.get(conf); + FileSystem fs = FileSystem.get(conf); Path path = new Path(filename); assertTrue("Trace file does not exist", fs.exists(path)); FSDataInputStream in = fs.open(path); @@ -128,15 +128,14 @@ public void testFilter(@Injectable final DrillbitContext bitContext, @Injectable wrap.readFromStream(in); VectorAccessible container = wrap.get(); - /* Assert there are no selection vectors */ + /* Assert there are no selection vectors */ assertTrue(wrap.getSv2() == null); - /* Assert there is only one record */ - assertTrue(container.getRecordCount() == 1); + /* Assert there is only one record */ + assertTrue(container.getRecordCount() == 1); - /* Read the Integer value and ASSERT its Integer.MIN_VALUE */ - int value = (int) container.iterator().next().getValueVector().getAccessor().getObject(0); - assertTrue(value == Integer.MIN_VALUE); + /* Read the Integer value and ASSERT its Integer.MIN_VALUE */ + final int value = (int) container.iterator().next().getValueVector().getAccessor().getObject(0); + assertEquals(Integer.MIN_VALUE, value); } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java index 07de27f394b..ff750416a63 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java @@ -27,7 +27,7 @@ import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.FragmentRoot; @@ -46,31 +46,27 @@ import com.google.common.io.Files; public class TestSimpleUnion extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleUnion.class); - DrillConfig c = DrillConfig.create(); - +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleUnion.class); + private final DrillConfig c = DrillConfig.create(); @Test public void testUnion(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{ - - new NonStrictExpectations(){{ bitContext.getMetrics(); result = new MetricRegistry(); - bitContext.getAllocator(); result = new TopLevelAllocator(); + bitContext.getAllocator(); result = new RootAllocator(c); bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c); bitContext.getConfig(); result = c; bitContext.getCompiler(); result = CodeCompiler.getTestCompiler(c); }}; - PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance()); PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/union/test1.json"), Charsets.UTF_8)); FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c); FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry); SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next())); - int[] counts = new int[]{100,50}; - int i=0; + int[] counts = new int[]{100, 50}; + int i = 0; while(exec.next()){ System.out.println("iteration count:" + exec.getRecordCount()); assertEquals(counts[i++], exec.getRecordCount()); @@ -80,7 +76,5 @@ public void testUnion(@Injectable final DrillbitContext bitContext, @Injectable throw context.getFailureCause(); } assertTrue(!context.isFailed()); - } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java index f4d505db499..fa3394a10f1 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java @@ -62,17 +62,14 @@ public void simpleCsv() throws Exception { fs.delete(path, true); } - String plan = Files.toString(FileUtils.getResourceAsFile("/writer/simple_csv_writer.json"), Charsets.UTF_8); - - List results = testPhysicalWithResults(plan); - - RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator()); - - QueryDataBatch batch = results.get(0); + final String plan = Files.toString(FileUtils.getResourceAsFile("/writer/simple_csv_writer.json"), Charsets.UTF_8); + final List results = testPhysicalWithResults(plan); + final RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator()); + final QueryDataBatch batch = results.get(0); assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData())); - VarCharVector fragmentIdV = (VarCharVector) batchLoader.getValueAccessorById(VarCharVector.class, 0).getValueVector(); - BigIntVector recordWrittenV = (BigIntVector) batchLoader.getValueAccessorById(BigIntVector.class, 1).getValueVector(); + final VarCharVector fragmentIdV = (VarCharVector) batchLoader.getValueAccessorById(VarCharVector.class, 0).getValueVector(); + final BigIntVector recordWrittenV = (BigIntVector) batchLoader.getValueAccessorById(BigIntVector.class, 1).getValueVector(); // expected only one row in output assertEquals(1, batchLoader.getRecordCount()); @@ -84,10 +81,10 @@ public void simpleCsv() throws Exception { assertTrue(fs.exists(path)); // expect two files - FileStatus[] fileStatuses = fs.globStatus(new Path(path.toString(), "*.csv")); + final FileStatus[] fileStatuses = fs.globStatus(new Path(path.toString(), "*.csv")); assertTrue(2 == fileStatuses.length); - for (QueryDataBatch b : results) { + for (final QueryDataBatch b : results) { b.release(); } batchLoader.clear(); diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java index f57e76503b9..ed893a919a6 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java @@ -23,6 +23,7 @@ import java.util.List; +import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.expression.ExpressionPosition; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.types.TypeProtos; @@ -30,12 +31,13 @@ import org.apache.drill.common.types.Types; import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.record.MaterializedField; import org.apache.drill.exec.record.RecordBatchLoader; import org.apache.drill.exec.record.VectorWrapper; import org.apache.drill.exec.record.WritableBatch; import org.apache.drill.exec.vector.AllocationHelper; +import org.apache.drill.exec.vector.BaseValueVector; import org.apache.drill.exec.vector.IntVector; import org.apache.drill.exec.vector.NullableVarCharVector; import org.apache.drill.exec.vector.ValueVector; @@ -45,31 +47,36 @@ import com.google.common.collect.Lists; public class TestLoad extends ExecTest { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestLoad.class); @Test public void testLoadValueVector() throws Exception { - BufferAllocator allocator = new TopLevelAllocator(); - ValueVector fixedV = new IntVector(MaterializedField.create(new SchemaPath("ints", ExpressionPosition.UNKNOWN), + final BufferAllocator allocator = new RootAllocator(DrillConfig.create()); + final ValueVector fixedV = new IntVector( + MaterializedField.create(new SchemaPath("ints", ExpressionPosition.UNKNOWN), Types.required(MinorType.INT)), allocator); - ValueVector varlenV = new VarCharVector(MaterializedField.create( + final ValueVector varlenV = new VarCharVector(MaterializedField.create( new SchemaPath("chars", ExpressionPosition.UNKNOWN), Types.required(MinorType.VARCHAR)), allocator); - ValueVector nullableVarlenV = new NullableVarCharVector(MaterializedField.create(new SchemaPath("chars", - ExpressionPosition.UNKNOWN), Types.optional(MinorType.VARCHAR)), allocator); + final ValueVector nullableVarlenV = new NullableVarCharVector( + MaterializedField.create(new SchemaPath("chars", ExpressionPosition.UNKNOWN), + Types.optional(MinorType.VARCHAR)), allocator); - List vectors = Lists.newArrayList(fixedV, varlenV, nullableVarlenV); - for (ValueVector v : vectors) { + final List vectors = Lists.newArrayList(fixedV, varlenV, nullableVarlenV); + for (final ValueVector v : vectors) { AllocationHelper.allocate(v, 100, 50); + BaseValueVector.checkBufRefs(v); v.getMutator().generateTestData(100); + BaseValueVector.checkBufRefs(v); } - WritableBatch writableBatch = WritableBatch.getBatchNoHV(100, vectors, false); - RecordBatchLoader batchLoader = new RecordBatchLoader(allocator); - ByteBuf[] byteBufs = writableBatch.getBuffers(); + final WritableBatch writableBatch = WritableBatch.getBatchNoHV(100, vectors, false); + final RecordBatchLoader batchLoader = new RecordBatchLoader(allocator); + final ByteBuf[] byteBufs = writableBatch.getBuffers(); int bytes = 0; - for (int i = 0; i < byteBufs.length; i++) { - bytes += byteBufs[i].writerIndex(); + for(final ByteBuf writableBuf : byteBufs) { + bytes += writableBuf.writerIndex(); } - DrillBuf byteBuf = allocator.buffer(bytes); + final DrillBuf byteBuf = allocator.buffer(bytes); int index = 0; for (int i = 0; i < byteBufs.length; i++) { byteBufs[i].readBytes(byteBuf, index, byteBufs[i].writerIndex()); @@ -96,15 +103,15 @@ public void testLoadValueVector() throws Exception { for (int r = 0; r < batchLoader.getRecordCount(); r++) { boolean first = true; recordCount++; - for (VectorWrapper v : batchLoader) { + for (final VectorWrapper v : batchLoader) { if (first) { first = false; } else { System.out.print("\t"); } - ValueVector.Accessor accessor = v.getValueVector().getAccessor(); + final ValueVector.Accessor accessor = v.getValueVector().getAccessor(); if (v.getField().getType().getMinorType() == TypeProtos.MinorType.VARCHAR) { - Object obj = accessor.getObject(r); + final Object obj = accessor.getObject(r); if (obj != null) { System.out.print(accessor.getObject(r)); } else { @@ -122,5 +129,4 @@ public void testLoadValueVector() throws Exception { batchLoader.clear(); writableBatch.clear(); } - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java index 4eeb8f282e0..f966121d7a8 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java @@ -19,11 +19,14 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import io.netty.buffer.DrillBuf; import java.nio.charset.Charset; import org.apache.drill.common.AutoCloseables; +import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.exception.OversizedAllocationException; @@ -35,16 +38,19 @@ import org.apache.drill.exec.expr.holders.NullableVar16CharHolder; import org.apache.drill.exec.expr.holders.NullableVarCharHolder; import org.apache.drill.exec.expr.holders.RepeatedFloat4Holder; +import org.apache.drill.exec.expr.holders.RepeatedIntHolder; import org.apache.drill.exec.expr.holders.RepeatedVarBinaryHolder; import org.apache.drill.exec.expr.holders.UInt4Holder; import org.apache.drill.exec.expr.holders.VarCharHolder; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.BufferAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.record.MaterializedField; import org.apache.drill.exec.vector.BaseValueVector; import org.apache.drill.exec.vector.BitVector; import org.apache.drill.exec.vector.NullableFloat4Vector; import org.apache.drill.exec.vector.NullableUInt4Vector; import org.apache.drill.exec.vector.NullableVarCharVector; +import org.apache.drill.exec.vector.RepeatedIntVector; import org.apache.drill.exec.vector.UInt4Vector; import org.apache.drill.exec.vector.ValueVector; import org.apache.drill.exec.vector.VarCharVector; @@ -56,25 +62,23 @@ import org.junit.Test; public class TestValueVector extends ExecTest { - private final static SchemaPath EMPTY_SCHEMA_PATH = SchemaPath.getSimplePath(""); + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestValueVector.class); - private final static byte[] STR1 = new String("AAAAA1").getBytes(Charset.forName("UTF-8")); - private final static byte[] STR2 = new String("BBBBBBBBB2").getBytes(Charset.forName("UTF-8")); - private final static byte[] STR3 = new String("CCCC3").getBytes(Charset.forName("UTF-8")); + private final static SchemaPath EMPTY_SCHEMA_PATH = SchemaPath.getSimplePath(""); - private TopLevelAllocator allocator; + private final static Charset utf8Charset = Charset.forName("UTF-8"); + private final static byte[] STR1 = new String("AAAAA1").getBytes(utf8Charset); + private final static byte[] STR2 = new String("BBBBBBBBB2").getBytes(utf8Charset); + private final static byte[] STR3 = new String("CCCC3").getBytes(utf8Charset); - @Before - public void init() { - allocator = new TopLevelAllocator(); - } + private final DrillConfig drillConfig = DrillConfig.create(); + private final RootAllocator allocator = new RootAllocator(drillConfig); @After - public void terminate() { + public void terminate() throws Exception { allocator.close(); } - @Test(expected = OversizedAllocationException.class) public void testFixedVectorReallocation() { final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE); @@ -163,9 +167,9 @@ public void testVariableVectorReallocation() { public void testFixedType() { MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE); - // Create a new value vector for 1024 integers - try (UInt4Vector vector = new UInt4Vector(field, allocator)) { - UInt4Vector.Mutator m = vector.getMutator(); + // Create a new value vector for 1024 integers. + try (final UInt4Vector vector = new UInt4Vector(field, allocator)) { + final UInt4Vector.Mutator m = vector.getMutator(); vector.allocateNew(1024); // Put and set a few values @@ -174,33 +178,36 @@ public void testFixedType() { m.setSafe(100, 102); m.setSafe(1022, 103); m.setSafe(1023, 104); - assertEquals(100, vector.getAccessor().get(0)); - assertEquals(101, vector.getAccessor().get(1)); - assertEquals(102, vector.getAccessor().get(100)); - assertEquals(103, vector.getAccessor().get(1022)); - assertEquals(104, vector.getAccessor().get(1023)); + + final UInt4Vector.Accessor accessor = vector.getAccessor(); + assertEquals(100, accessor.get(0)); + assertEquals(101, accessor.get(1)); + assertEquals(102, accessor.get(100)); + assertEquals(103, accessor.get(1022)); + assertEquals(104, accessor.get(1023)); } } @Test public void testNullableVarLen2() { - MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVarCharHolder.TYPE); + final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVarCharHolder.TYPE); - // Create a new value vector for 1024 integers - try (NullableVarCharVector vector = new NullableVarCharVector(field, allocator)) { - NullableVarCharVector.Mutator m = vector.getMutator(); + // Create a new value vector for 1024 integers. + try (final NullableVarCharVector vector = new NullableVarCharVector(field, allocator)) { + final NullableVarCharVector.Mutator m = vector.getMutator(); vector.allocateNew(1024 * 10, 1024); m.set(0, STR1); m.set(1, STR2); m.set(2, STR3); - // Check the sample strings - assertArrayEquals(STR1, vector.getAccessor().get(0)); - assertArrayEquals(STR2, vector.getAccessor().get(1)); - assertArrayEquals(STR3, vector.getAccessor().get(2)); + // Check the sample strings. + final NullableVarCharVector.Accessor accessor = vector.getAccessor(); + assertArrayEquals(STR1, accessor.get(0)); + assertArrayEquals(STR2, accessor.get(1)); + assertArrayEquals(STR3, accessor.get(2)); - // Ensure null value throws + // Ensure null value throws. boolean b = false; try { vector.getAccessor().get(3); @@ -212,14 +219,187 @@ public void testNullableVarLen2() { } } + private static DrillBuf combineBuffers(final BufferAllocator allocator, final DrillBuf[] buffers) { + // find the total size we'll need + int size = 0; + for(final DrillBuf buffer : buffers) { + size += buffer.readableBytes(); + } + + // create the new buffer + final DrillBuf newBuf = allocator.buffer(size); + final DrillBuf writeBuf = newBuf; + for(final DrillBuf buffer : buffers) { + final DrillBuf readBuf = (DrillBuf) buffer.slice(); + final int nBytes = readBuf.readableBytes(); + for(int i = 0; i < nBytes; ++i) { + writeBuf.writeByte(readBuf.readByte()); + } + } + + return newBuf; + } + + @Test + public void testRepeatedIntVector() { + final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, RepeatedIntHolder.TYPE); + + // Create a new value vector. + final RepeatedIntVector vector1 = new RepeatedIntVector(field, allocator); + + // Populate the vector. + final int[] values = {2, 3, 5, 7, 11, 13, 17, 19, 23, 27}; // some tricksy primes + final int nRecords = 7; + final int nElements = values.length; + vector1.allocateNew(nRecords, nRecords * nElements); + final RepeatedIntVector.Mutator mutator = vector1.getMutator(); + for(int recordIndex = 0; recordIndex < nRecords; ++recordIndex) { + mutator.startNewValue(recordIndex); + for(int elementIndex = 0; elementIndex < nElements; ++elementIndex) { + mutator.add(recordIndex, recordIndex * values[elementIndex]); + } + } + mutator.setValueCount(nRecords); + + // Verify the contents. + final RepeatedIntVector.Accessor accessor1 = vector1.getAccessor(); + assertEquals(nRecords, accessor1.getValueCount()); + for(int recordIndex = 0; recordIndex < nRecords; ++recordIndex) { + for(int elementIndex = 0; elementIndex < nElements; ++elementIndex) { + final int value = accessor1.get(recordIndex, elementIndex); + assertEquals(recordIndex * values[elementIndex], value); + } + } + +/* TODO(cwestin) +the interface to load has changed + // Serialize, reify, and verify. + final DrillBuf[] buffers1 = vector1.getBuffers(false); + final DrillBuf buffer1 = combineBuffers(allocator, buffers1); + final RepeatedIntVector vector2 = new RepeatedIntVector(field, allocator); + vector2.load(nRecords, nRecords * nElements, buffer1); + + final RepeatedIntVector.Accessor accessor2 = vector2.getAccessor(); + for(int recordIndex = 0; recordIndex < nRecords; ++recordIndex) { + for(int elementIndex = 0; elementIndex < nElements; ++elementIndex) { + final int value = accessor2.get(recordIndex, elementIndex); + assertEquals(accessor1.get(recordIndex, elementIndex), value); + } + } +*/ + + vector1.close(); +/* TODO(cwestin) + vector2.close(); + buffer1.release(); +*/ + } + + @Test + public void testVarCharVectorLoad() { + final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, VarCharHolder.TYPE); + + // Create a new value vector for 1024 variable length strings. + final VarCharVector vector1 = new VarCharVector(field, allocator); + final VarCharVector.Mutator mutator = vector1.getMutator(); + vector1.allocateNew(1024 * 10, 1024); + + // Populate the vector. + final StringBuilder stringBuilder = new StringBuilder(); + final int valueCount = 10; + for(int i = 0; i < valueCount; ++i) { + stringBuilder.append('x'); + mutator.setSafe(i, stringBuilder.toString().getBytes(utf8Charset)); + } + mutator.setValueCount(valueCount); + assertEquals(valueCount, vector1.getAccessor().getValueCount()); + + // Combine the backing buffers so we can load them into a new vector. + final DrillBuf[] buffers1 = vector1.getBuffers(false); + final DrillBuf buffer1 = combineBuffers(allocator, buffers1); + final VarCharVector vector2 = new VarCharVector(field, allocator); + vector2.load(buffer1.readableBytes(), valueCount, buffer1); + + // Check the contents of the new vector. + final VarCharVector.Accessor accessor = vector2.getAccessor(); + stringBuilder.setLength(0); + for(int i = 0; i < valueCount; ++i) { + stringBuilder.append('x'); + final Object object = accessor.getObject(i); + System.out.println(object); + assertEquals(stringBuilder.toString(), object.toString()); + } + + vector1.close(); + vector2.close(); + buffer1.release(); + } + + @Test + public void testNullableVarCharVectorLoad() { + final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVarCharHolder.TYPE); + + // Create a new value vector for 1024 nullable variable length strings. + final NullableVarCharVector vector1 = new NullableVarCharVector(field, allocator); + final NullableVarCharVector.Mutator mutator = vector1.getMutator(); + vector1.allocateNew(1024 * 10, 1024); + + // Populate the vector. + final StringBuilder stringBuilder = new StringBuilder(); + final int valueCount = 10; + for(int i = 0; i < valueCount; ++i) { + stringBuilder.append('x'); + mutator.set(i, stringBuilder.toString().getBytes(utf8Charset)); + } + + // Check the contents. + final NullableVarCharVector.Accessor accessor1 = vector1.getAccessor(); + stringBuilder.setLength(0); + for(int i = 0; i < valueCount; ++i) { + stringBuilder.append('x'); + final Object object = accessor1.getObject(i); + assertEquals(stringBuilder.toString(), object.toString()); + } + + mutator.setValueCount(valueCount); + assertEquals(valueCount, vector1.getAccessor().getValueCount()); + + // Still ok after setting value count? + stringBuilder.setLength(0); + for(int i = 0; i < valueCount; ++i) { + stringBuilder.append('x'); + final Object object = accessor1.getObject(i); + System.out.println(object); + assertEquals(stringBuilder.toString(), object.toString()); + } + + // Combine into a single buffer so we can load it into a new vector. + final DrillBuf[] buffers1 = vector1.getBuffers(false); + final DrillBuf buffer1 = combineBuffers(allocator, buffers1); + final NullableVarCharVector vector2 = new NullableVarCharVector(field, allocator); + vector2.load(buffer1.readableBytes(), valueCount, buffer1); + + // Check the vector's contents. + final NullableVarCharVector.Accessor accessor2 = vector2.getAccessor(); + stringBuilder.setLength(0); + for(int i = 0; i < valueCount; ++i) { + stringBuilder.append('x'); + final Object object = accessor2.getObject(i); + assertEquals(stringBuilder.toString(), object.toString()); + } + + vector1.close(); + vector2.close(); + buffer1.release(); + } @Test public void testNullableFixedType() { - MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableUInt4Holder.TYPE); + final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableUInt4Holder.TYPE); - // Create a new value vector for 1024 integers - try (NullableUInt4Vector vector = new NullableUInt4Vector(field, allocator)) { - NullableUInt4Vector.Mutator m = vector.getMutator(); + // Create a new value vector for 1024 integers. + try (final NullableUInt4Vector vector = new NullableUInt4Vector(field, allocator)) { + final NullableUInt4Vector.Mutator m = vector.getMutator(); vector.allocateNew(1024); // Put and set a few values @@ -228,17 +408,19 @@ public void testNullableFixedType() { m.set(100, 102); m.set(1022, 103); m.set(1023, 104); - assertEquals(100, vector.getAccessor().get(0)); - assertEquals(101, vector.getAccessor().get(1)); - assertEquals(102, vector.getAccessor().get(100)); - assertEquals(103, vector.getAccessor().get(1022)); - assertEquals(104, vector.getAccessor().get(1023)); + + final NullableUInt4Vector.Accessor accessor = vector.getAccessor(); + assertEquals(100, accessor.get(0)); + assertEquals(101, accessor.get(1)); + assertEquals(102, accessor.get(100)); + assertEquals(103, accessor.get(1022)); + assertEquals(104, accessor.get(1023)); // Ensure null values throw { boolean b = false; try { - vector.getAccessor().get(3); + accessor.get(3); } catch (IllegalStateException e) { b = true; } finally { @@ -246,12 +428,11 @@ public void testNullableFixedType() { } } - vector.allocateNew(2048); { boolean b = false; try { - vector.getAccessor().get(0); + accessor.get(0); } catch (IllegalStateException e) { b = true; } finally { @@ -264,14 +445,13 @@ public void testNullableFixedType() { m.set(100, 102); m.set(1022, 103); m.set(1023, 104); - assertEquals(100, vector.getAccessor().get(0)); - assertEquals(101, vector.getAccessor().get(1)); - assertEquals(102, vector.getAccessor().get(100)); - assertEquals(103, vector.getAccessor().get(1022)); - assertEquals(104, vector.getAccessor().get(1023)); - - // Ensure null values throw + assertEquals(100, accessor.get(0)); + assertEquals(101, accessor.get(1)); + assertEquals(102, accessor.get(100)); + assertEquals(103, accessor.get(1022)); + assertEquals(104, accessor.get(1023)); + // Ensure null values throw. { boolean b = false; try { @@ -283,31 +463,32 @@ public void testNullableFixedType() { } } } - } @Test public void testNullableFloat() { - MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableFloat4Holder.TYPE); + final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableFloat4Holder.TYPE); // Create a new value vector for 1024 integers - try (NullableFloat4Vector vector = (NullableFloat4Vector) TypeHelper.getNewVector(field, allocator)) { - NullableFloat4Vector.Mutator m = vector.getMutator(); + try (final NullableFloat4Vector vector = (NullableFloat4Vector) TypeHelper.getNewVector(field, allocator)) { + final NullableFloat4Vector.Mutator m = vector.getMutator(); vector.allocateNew(1024); - // Put and set a few values + // Put and set a few values. m.set(0, 100.1f); m.set(1, 101.2f); m.set(100, 102.3f); m.set(1022, 103.4f); m.set(1023, 104.5f); - assertEquals(100.1f, vector.getAccessor().get(0), 0); - assertEquals(101.2f, vector.getAccessor().get(1), 0); - assertEquals(102.3f, vector.getAccessor().get(100), 0); - assertEquals(103.4f, vector.getAccessor().get(1022), 0); - assertEquals(104.5f, vector.getAccessor().get(1023), 0); - // Ensure null values throw + final NullableFloat4Vector.Accessor accessor = vector.getAccessor(); + assertEquals(100.1f, accessor.get(0), 0); + assertEquals(101.2f, accessor.get(1), 0); + assertEquals(102.3f, accessor.get(100), 0); + assertEquals(103.4f, accessor.get(1022), 0); + assertEquals(104.5f, accessor.get(1023), 0); + + // Ensure null values throw. { boolean b = false; try { @@ -323,7 +504,7 @@ public void testNullableFloat() { { boolean b = false; try { - vector.getAccessor().get(0); + accessor.get(0); } catch (IllegalStateException e) { b = true; } finally { @@ -335,11 +516,11 @@ public void testNullableFloat() { @Test public void testBitVector() { - MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, BitHolder.TYPE); + final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, BitHolder.TYPE); // Create a new value vector for 1024 integers - try (BitVector vector = new BitVector(field, allocator)) { - BitVector.Mutator m = vector.getMutator(); + try (final BitVector vector = new BitVector(field, allocator)) { + final BitVector.Mutator m = vector.getMutator(); vector.allocateNew(1024); // Put and set a few values @@ -347,38 +528,39 @@ public void testBitVector() { m.set(1, 0); m.set(100, 0); m.set(1022, 1); - assertEquals(1, vector.getAccessor().get(0)); - assertEquals(0, vector.getAccessor().get(1)); - assertEquals(0, vector.getAccessor().get(100)); - assertEquals(1, vector.getAccessor().get(1022)); + + final BitVector.Accessor accessor = vector.getAccessor(); + assertEquals(1, accessor.get(0)); + assertEquals(0, accessor.get(1)); + assertEquals(0, accessor.get(100)); + assertEquals(1, accessor.get(1022)); // test setting the same value twice m.set(0, 1); m.set(0, 1); m.set(1, 0); m.set(1, 0); - assertEquals(1, vector.getAccessor().get(0)); - assertEquals(0, vector.getAccessor().get(1)); + assertEquals(1, accessor.get(0)); + assertEquals(0, accessor.get(1)); // test toggling the values m.set(0, 0); m.set(1, 1); - assertEquals(0, vector.getAccessor().get(0)); - assertEquals(1, vector.getAccessor().get(1)); + assertEquals(0, accessor.get(0)); + assertEquals(1, accessor.get(1)); // Ensure unallocated space returns 0 - assertEquals(0, vector.getAccessor().get(3)); + assertEquals(0, accessor.get(3)); } } - @Test - public void testReAllocNullableFixedWidthVector() throws Exception { - MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableFloat4Holder.TYPE); + public void testReAllocNullableFixedWidthVector() { + final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableFloat4Holder.TYPE); // Create a new value vector for 1024 integers - try (NullableFloat4Vector vector = (NullableFloat4Vector) TypeHelper.getNewVector(field, allocator)) { - NullableFloat4Vector.Mutator m = vector.getMutator(); + try (final NullableFloat4Vector vector = (NullableFloat4Vector) TypeHelper.getNewVector(field, allocator)) { + final NullableFloat4Vector.Mutator m = vector.getMutator(); vector.allocateNew(1024); assertEquals(1024, vector.getValueCapacity()); @@ -394,11 +576,11 @@ public void testReAllocNullableFixedWidthVector() throws Exception { // Check valueCapacity is more than initial allocation assertEquals(1024 * 2, vector.getValueCapacity()); - assertEquals(100.1f, vector.getAccessor().get(0), 0); - assertEquals(102.3f, vector.getAccessor().get(100), 0); - assertEquals(104.5f, vector.getAccessor().get(1023), 0); - assertEquals(105.5f, vector.getAccessor().get(2000), 0); - + final NullableFloat4Vector.Accessor accessor = vector.getAccessor(); + assertEquals(100.1f, accessor.get(0), 0); + assertEquals(102.3f, accessor.get(100), 0); + assertEquals(104.5f, accessor.get(1023), 0); + assertEquals(105.5f, accessor.get(2000), 0); // Set the valueCount to be more than valueCapacity of current allocation. This is possible for NullableValueVectors // as we don't call setSafe for null values, but we do call setValueCount when all values are inserted into the @@ -408,12 +590,12 @@ public void testReAllocNullableFixedWidthVector() throws Exception { } @Test - public void testReAllocNullableVariableWidthVector() throws Exception { - MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVarCharHolder.TYPE); + public void testReAllocNullableVariableWidthVector() { + final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVarCharHolder.TYPE); // Create a new value vector for 1024 integers - try (NullableVarCharVector vector = (NullableVarCharVector) TypeHelper.getNewVector(field, allocator)) { - NullableVarCharVector.Mutator m = vector.getMutator(); + try (final NullableVarCharVector vector = (NullableVarCharVector) TypeHelper.getNewVector(field, allocator)) { + final NullableVarCharVector.Mutator m = vector.getMutator(); vector.allocateNew(); int initialCapacity = vector.getValueCapacity(); @@ -428,9 +610,10 @@ public void testReAllocNullableVariableWidthVector() throws Exception { // Check valueCapacity is more than initial allocation assertEquals((initialCapacity + 1) * 2 - 1, vector.getValueCapacity()); - assertArrayEquals(STR1, vector.getAccessor().get(0)); - assertArrayEquals(STR2, vector.getAccessor().get(initialCapacity - 1)); - assertArrayEquals(STR3, vector.getAccessor().get(initialCapacity + 200)); + final NullableVarCharVector.Accessor accessor = vector.getAccessor(); + assertArrayEquals(STR1, accessor.get(0)); + assertArrayEquals(STR2, accessor.get(initialCapacity - 1)); + assertArrayEquals(STR3, accessor.get(initialCapacity + 200)); // Set the valueCount to be more than valueCapacity of current allocation. This is possible for NullableValueVectors // as we don't call setSafe for null values, but we do call setValueCount when the current batch is processed. diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java index 4230518e1f9..a32322b3048 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java @@ -18,12 +18,11 @@ package org.apache.drill.exec.server; import org.apache.drill.exec.client.DrillClient; +import org.apache.drill.exec.memory.OutOfMemoryException; import org.glassfish.hk2.api.Factory; -public class DrillClientFactory implements Factory{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClientFactory.class); - - +public class DrillClientFactory implements Factory { +// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClientFactory.class); @Override public void dispose(DrillClient arg0) { @@ -31,8 +30,10 @@ public void dispose(DrillClient arg0) { @Override public DrillClient provide() { - return new DrillClient(); + try { + return new DrillClient(); + } catch(OutOfMemoryException e) { + throw new RuntimeException(e); + } } - - } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java index ce09f687241..ea7b4cb8a1d 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java @@ -36,6 +36,7 @@ import org.apache.drill.SingleRowListener; import org.apache.drill.common.AutoCloseables; import org.apache.drill.common.concurrent.ExtendedLatch; +import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.exceptions.UserException; import org.apache.drill.common.types.TypeProtos.MinorType; @@ -46,7 +47,7 @@ import org.apache.drill.exec.exception.DrillbitStartupException; import org.apache.drill.exec.exception.SchemaChangeException; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.physical.impl.ScreenCreator; import org.apache.drill.exec.physical.impl.SingleSenderCreator.SingleSenderRootExec; import org.apache.drill.exec.physical.impl.mergereceiver.MergingRecordBatch; @@ -119,7 +120,6 @@ private static void startDrillbit(final String name, final RemoteServiceSet remo } try { - @SuppressWarnings("resource") final Drillbit drillbit = Drillbit.start(zkHelper.getConfig(), remoteServiceSet); drillbits.put(name, drillbit); } catch (final DrillbitStartupException e) { @@ -133,7 +133,6 @@ private static void startDrillbit(final String name, final RemoteServiceSet remo * @param name name of the drillbit */ private static void stopDrillbit(final String name) { - @SuppressWarnings("resource") final Drillbit drillbit = drillbits.get(name); if (drillbit == null) { throw new IllegalStateException("No Drillbit named \"" + name + "\" found"); @@ -234,7 +233,7 @@ private static void clearAllInjections() { */ private static void assertDrillbitsOk() { final SingleRowListener listener = new SingleRowListener() { - private final BufferAllocator bufferAllocator = new TopLevelAllocator(zkHelper.getConfig()); + private final BufferAllocator bufferAllocator = new RootAllocator(zkHelper.getConfig()); private final RecordBatchLoader loader = new RecordBatchLoader(bufferAllocator); @Override @@ -273,7 +272,7 @@ public void rowArrived(final QueryDataBatch queryResultBatch) { @Override public void cleanup() { - bufferAllocator.close(); + DrillAutoCloseables.closeNoChecked(bufferAllocator); } }; diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestDirectCodecFactory.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestDirectCodecFactory.java index 644144e18a6..712b99a59aa 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestDirectCodecFactory.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestDirectCodecFactory.java @@ -23,9 +23,10 @@ import java.util.Random; import org.apache.drill.common.DeferredException; +import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.ExecTest; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.store.parquet.DirectCodecFactory; import org.apache.drill.exec.store.parquet.DirectCodecFactory.ByteBufBytesInput; import org.apache.drill.exec.store.parquet.DirectCodecFactory.DirectBytesDecompressor; @@ -38,16 +39,17 @@ import parquet.hadoop.metadata.CompressionCodecName; public class TestDirectCodecFactory extends ExecTest { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestDirectCodecFactory.class); + // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestDirectCodecFactory.class); private static enum Decompression { ON_HEAP, OFF_HEAP, DRILLBUF } - private void test(int size, CompressionCodecName codec, boolean useOnHeapCompression, Decompression decomp) { + private void test(int size, CompressionCodecName codec, boolean useOnHeapCompression, Decompression decomp) throws Exception { DrillBuf rawBuf = null; DrillBuf outBuf = null; - try (BufferAllocator allocator = new TopLevelAllocator(); + final DrillConfig drillConfig = DrillConfig.create(); + try (BufferAllocator allocator = new RootAllocator(drillConfig); DirectCodecFactory codecFactory = new DirectCodecFactory(new Configuration(), allocator)) { try { rawBuf = allocator.buffer(size); diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/TestJsonRecordReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/TestJsonRecordReader.java index bb1af9eb2e6..38a92db874d 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/TestJsonRecordReader.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/TestJsonRecordReader.java @@ -26,11 +26,11 @@ import static org.junit.Assert.assertTrue; -public class TestJsonRecordReader extends BaseTestQuery{ - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestJsonRecordReader.class); +public class TestJsonRecordReader extends BaseTestQuery { + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestJsonRecordReader.class); @Test - public void testComplexJsonInput() throws Exception{ + public void testComplexJsonInput() throws Exception { // test("select z[0]['orange'] from cp.`jsoninput/input2.json` limit 10"); test("select `integer`, x['y'] as x1, x['y'] as x2, z[0], z[0]['orange'], z[1]['pink'] from cp.`jsoninput/input2.json` limit 10 "); // test("select x from cp.`jsoninput/input2.json`"); @@ -44,14 +44,14 @@ public void testContainingArray() throws Exception { } @Test - public void testComplexMultipleTimes() throws Exception{ - for(int i =0 ; i < 5; i++){ + public void testComplexMultipleTimes() throws Exception { + for(int i =0 ; i < 5; i++) { test("select * from cp.`join/merge_join.json`"); } } @Test - public void trySimpleQueryWithLimit() throws Exception{ + public void trySimpleQueryWithLimit() throws Exception { test("select * from cp.`limit/test1.json` limit 10"); } @@ -89,9 +89,7 @@ public void testExceptionHandling() throws Exception { @Test //DRILL-1832 public void testJsonWithNulls1() throws Exception { - final String query="select * from cp.`jsoninput/twitter_43.json`"; - testBuilder() .sqlQuery(query) .unOrdered() @@ -101,9 +99,7 @@ public void testJsonWithNulls1() throws Exception { @Test //DRILL-1832 public void testJsonWithNulls2() throws Exception { - final String query="select SUM(1) as `sum_Number_of_Records_ok` from cp.`/jsoninput/twitter_43.json` having (COUNT(1) > 0)"; - testBuilder() .sqlQuery(query) .unOrdered() diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/FieldInfo.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/FieldInfo.java index 98313bcfdd1..3beb11fcb66 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/FieldInfo.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/FieldInfo.java @@ -40,7 +40,6 @@ public class FieldInfo { throw new IllegalStateException("generator is designed to use 3 values"); } this.values = values; - this.type = type; } } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java index 61380cf09bc..88259d6d8f7 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java @@ -41,7 +41,7 @@ import org.apache.drill.exec.exception.SchemaChangeException; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; import org.apache.drill.exec.memory.BufferAllocator; -import org.apache.drill.exec.memory.TopLevelAllocator; +import org.apache.drill.exec.memory.RootAllocator; import org.apache.drill.exec.ops.FragmentContext; import org.apache.drill.exec.physical.impl.OutputMutator; import org.apache.drill.exec.proto.BitControl; @@ -380,7 +380,6 @@ private void validateFooters(final List