aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--common/src/main/java/org/apache/drill/common/config/DrillConfig.java26
-rw-r--r--exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java2
-rw-r--r--exec/java-exec/pom.xml36
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java7
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMap.java10
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMultiMap.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java62
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java64
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java5
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java8
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java4
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java10
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java8
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/memory/AccountingByteBuf.java859
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java190
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java118
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java42
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java65
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryException.java51
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java162
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java9
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java155
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java62
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java26
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java13
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java8
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java4
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java36
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SendingAccountor.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java)53
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java21
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java11
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java18
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java9
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatchBuilder.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java9
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java429
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java51
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java31
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java43
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java36
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java7
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate4.java10
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java34
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java5
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/proto/helper/QueryIdHelper.java (renamed from protocol/src/main/java/org/apache/drill/exec/proto/helper/QueryIdHelper.java)0
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java11
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java10
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java4
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java12
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java4
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java17
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java20
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java23
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java25
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java73
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/FutureBitCommand.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/FutureBitCommand.java)15
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ListeningCommand.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListeningBitCommand.java)18
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ProtobufLengthDecoder.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java)26
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ReconnectingConnection.java242
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java88
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ResettableBarrier.java85
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java30
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcCommand.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCommand.java)6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java11
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java99
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java189
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java59
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/AvailabilityListener.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java)4
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ConnectionManagerRegistry.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ConnectionManagerRegistry.java)30
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java105
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlCommand.java28
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnection.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java)83
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnectionManager.java60
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlProtobufLengthDecoder.java40
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlRpcConfig.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java)20
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java)57
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java)87
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/Controller.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java)22
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java)41
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/DefaultInstanceHandler.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java)16
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/SendProgress.java22
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/WorkEventBus.java113
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/BitServerConnection.java55
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java100
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClientConnection.java80
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java63
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionManager.java53
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataDefaultInstanceHandler.java58
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataProtobufLengthDecoder.java38
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandler.java31
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandlerImpl.java64
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataRpcConfig.java42
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java128
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java94
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/SendProgress.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/SendProgress.java)2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java18
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserProtobufLengthDecoder.java38
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java25
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java9
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java31
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java33
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/util/AssertionUtil.java35
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java1
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/ErrorHelper.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java)2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java3
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java49
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java)14
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java211
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlHandlerImpl.java150
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java)14
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/DataCollector.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java)12
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java28
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java27
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java33
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java46
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java15
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java)51
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RootStatusReporter.java39
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java)16
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java)42
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java)23
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java)34
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootStatusReporter.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java)17
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java)23
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java)9
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java4
-rw-r--r--exec/java-exec/src/main/resources/drill-module.conf9
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/client/DumpCatTest.java11
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestRepeatedFunction.java15
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestEndianess.java4
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java9
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java38
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java6
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestImplicitCastFunctions.java27
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java177
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFunctions.java22
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java6
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java10
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java18
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestSimpleLimit.java12
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java2
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java15
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/project/TestSimpleProjection.java6
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSimpleSort.java10
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java6
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceMultiRecordBatch.java6
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/trace/TestTraceOutputDump.java6
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java6
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java6
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java12
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java4
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java148
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java14
-rw-r--r--exec/java-exec/src/test/resources/logback.xml13
-rw-r--r--protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java5438
-rw-r--r--protocol/src/main/java/org/apache/drill/exec/proto/BitData.java2228
-rw-r--r--protocol/src/main/java/org/apache/drill/exec/proto/CoordinationProtos.java239
-rw-r--r--protocol/src/main/java/org/apache/drill/exec/proto/ExecProtos.java6366
-rw-r--r--protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java432
-rw-r--r--protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java181
-rw-r--r--protocol/src/main/protobuf/BitControl.proto83
-rw-r--r--protocol/src/main/protobuf/BitData.proto34
-rw-r--r--protocol/src/main/protobuf/Coordination.proto5
-rw-r--r--protocol/src/main/protobuf/ExecutionProtos.proto83
-rw-r--r--protocol/src/main/protobuf/User.proto5
-rw-r--r--protocol/src/main/protobuf/UserBitShared.proto35
-rw-r--r--sqlparser/src/main/java/org/apache/drill/optiq/DrillOptiq.java2
185 files changed, 13351 insertions, 8918 deletions
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 ffbf910c4..d81ef79b7 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
@@ -17,6 +17,8 @@
*/
package org.apache.drill.common.config;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
import java.net.URL;
import java.util.Collection;
import java.util.List;
@@ -35,6 +37,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
@@ -42,13 +45,18 @@ public final class DrillConfig extends NestedConfig{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillConfig.class);
private final ObjectMapper mapper;
+ private final ImmutableList<String> startupArguments;
+ @SuppressWarnings("restriction") private static final long MAX_DIRECT_MEMORY = sun.misc.VM.maxDirectMemory();
@SuppressWarnings("unchecked")
private volatile List<Queue<Object>> sinkQueues = new CopyOnWriteArrayList<Queue<Object>>(new Queue[1]);
+
+ @SuppressWarnings("restriction")
@VisibleForTesting
public DrillConfig(Config config) {
super(config);
+
mapper = new ObjectMapper();
SimpleModule deserModule = new SimpleModule("LogicalExpressionDeserializationModule")
.addDeserializer(LogicalExpression.class, new LogicalExpression.De(this));
@@ -61,8 +69,16 @@ public final class DrillConfig extends NestedConfig{
mapper.registerSubtypes(LogicalOperatorBase.getSubTypes(this));
mapper.registerSubtypes(StorageEngineConfigBase.getSubTypes(this));
+ RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
+ this.startupArguments = ImmutableList.copyOf(bean.getInputArguments());
};
+
+
+
+ public List<String> getStartupArguments(){
+ return startupArguments;
+ }
/**
* Create a DrillConfig object using the default config file name
@@ -155,4 +171,14 @@ public final class DrillConfig extends NestedConfig{
public String toString(){
return this.root().render();
}
+
+ public static void main(String[] args) throws Exception{
+ //"-XX:MaxDirectMemorySize"
+ DrillConfig config = DrillConfig.create();
+
+ }
+
+ public static long getMaxDirectMemory(){
+ return MAX_DIRECT_MEMORY;
+ }
}
diff --git a/exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java b/exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java
index e5e1d20a5..efc13b5e5 100644
--- a/exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java
+++ b/exec/bufferl/src/main/java/io/netty/buffer/PooledUnsafeDirectByteBufL.java
@@ -30,7 +30,7 @@ import java.nio.channels.ClosedChannelException;
import java.nio.channels.GatheringByteChannel;
import java.nio.channels.ScatteringByteChannel;
-final class PooledUnsafeDirectByteBufL extends PooledByteBufL<ByteBuffer> {
+public final class PooledUnsafeDirectByteBufL extends PooledByteBufL<ByteBuffer> {
private static final boolean NATIVE_ORDER = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN;
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 2ffdae9e1..1c4dc328f 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -1,21 +1,16 @@
<?xml version="1.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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>exec-parent</artifactId>
@@ -37,6 +32,11 @@
<version>4.1</version>
</dependency>
<dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-pool2</artifactId>
+ <version>2.1</version>
+ </dependency>
+ <dependency>
<groupId>com.thoughtworks.paranamer</groupId>
<artifactId>paranamer</artifactId>
<version>2.5.6</version>
@@ -273,7 +273,7 @@
</resource>
<resource>
<directory>target/generated-sources</directory>
-<!-- <include>*/org</include> -->
+ <!-- <include>*/org</include> -->
<filtering>true</filtering>
</resource>
</resources>
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 465813c43..bcc113f0f 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
@@ -47,8 +47,9 @@ public interface ExecConstants {
public static final String TRACE_DUMP_FILESYSTEM = "drill.exec.trace.filesystem";
public static final String TEMP_DIRECTORIES = "drill.exec.tmp.directories";
public static final String TEMP_FILESYSTEM = "drill.exec.tmp.filesystem";
- public static final String SPOOLING_BUFFER_IMPL = "drill.exec.spooling.impl";
- public static final String SPOOLING_BUFFER_DELETE = "drill.exec.spooling.delete";
- public static final String SPOOLING_BUFFER_MEMORY = "drill.exec.spooling.size";
+ public static final String INCOMING_BUFFER_IMPL = "drill.exec.buffer.impl";
+ public static final String INCOMING_BUFFER_SIZE = "drill.exec.buffer.size"; // incoming buffer size (number of batches)
+ public static final String SPOOLING_BUFFER_DELETE = "drill.exec.buffer.spooling.delete";
+ public static final String SPOOLING_BUFFER_MEMORY = "drill.exec.buffer.spooling.size";
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
index aed6cc235..fdace08ca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
@@ -20,8 +20,8 @@ package org.apache.drill.exec.cache;
import java.io.Closeable;
import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
public interface DistributedCache extends Closeable{
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMap.java
index b7595f91a..8ea9cd19f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMap.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMap.java
@@ -19,10 +19,10 @@ package org.apache.drill.exec.cache;
import java.util.concurrent.TimeUnit;
-public interface DistributedMap<V> {
+public interface DistributedMap<V extends DrillSerializable> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedMap.class);
- public DrillSerializable get(String key);
- public void put(String key, DrillSerializable value);
- public void putIfAbsent(String key, DrillSerializable value);
- public void putIfAbsent(String key, DrillSerializable value, long ttl, TimeUnit timeUnit);
+ public V get(String key);
+ public void put(String key, V value);
+ public void putIfAbsent(String key, V value);
+ public void putIfAbsent(String key, V value, long ttl, TimeUnit timeUnit);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMultiMap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMultiMap.java
index 886f122d7..645fb7c0e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMultiMap.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedMultiMap.java
@@ -19,8 +19,8 @@ package org.apache.drill.exec.cache;
import java.util.Collection;
-public interface DistributedMultiMap<V> {
+public interface DistributedMultiMap<V extends DrillSerializable> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedMultiMap.class);
- public Collection<DrillSerializable> get(String key);
- public void put(String key, DrillSerializable value);
+ public Collection<V> get(String key);
+ public void put(String key, V value);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
index 9dd437307..2f0ab2ea4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
@@ -22,23 +22,28 @@ import java.util.Collection;
import java.util.List;
import java.util.concurrent.TimeUnit;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.hazelcast.config.SerializerConfig;
-import com.hazelcast.core.*;
import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.cache.ProtoBufImpl.HWorkQueueStatus;
import org.apache.drill.exec.cache.ProtoBufImpl.HandlePlan;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.WorkQueueStatus;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.Lists;
import com.hazelcast.config.Config;
-import org.apache.drill.exec.server.DrillbitContext;
+import com.hazelcast.config.SerializerConfig;
+import com.hazelcast.core.DuplicateInstanceNameException;
+import com.hazelcast.core.Hazelcast;
+import com.hazelcast.core.HazelcastInstance;
+import com.hazelcast.core.IAtomicLong;
+import com.hazelcast.core.IMap;
+import com.hazelcast.core.ITopic;
+import com.hazelcast.core.Message;
+import com.hazelcast.core.MessageListener;
public class HazelCache implements DistributedCache {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HazelCache.class);
@@ -67,8 +72,9 @@ public class HazelCache implements DistributedCache {
public void run() {
Config c = new Config();
- SerializerConfig sc = new SerializerConfig().setImplementation(new HCVectorAccessibleSerializer(allocator))
- .setTypeClass(VectorAccessibleSerializable.class);
+ SerializerConfig sc = new SerializerConfig() //
+ .setImplementation(new HCVectorAccessibleSerializer(allocator)) //
+ .setTypeClass(VectorAccessibleSerializable.class);
c.setInstanceName(instanceName);
c.getSerializationConfig().addSerializerConfig(sc);
instance = getInstanceOrCreateNew(c);
@@ -119,12 +125,14 @@ public class HazelCache implements DistributedCache {
@Override
public <V extends DrillSerializable> DistributedMultiMap<V> getMultiMap(Class<V> clazz) {
- return new HCDistributedMultiMapImpl(this.instance.getMultiMap(clazz.toString()), clazz);
+ com.hazelcast.core.MultiMap<String, V> mmap = this.instance.getMultiMap(clazz.toString());
+ return new HCDistributedMultiMapImpl<V>(mmap, clazz);
}
@Override
public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
- return new HCDistributedMapImpl(this.instance.getMap(clazz.toString()), clazz);
+ IMap<String, V> imap = this.instance.getMap(clazz.toString());
+ return new HCDistributedMapImpl<V>(imap, clazz);
}
@Override
@@ -132,51 +140,47 @@ public class HazelCache implements DistributedCache {
return new HCCounterImpl(this.instance.getAtomicLong(name));
}
- public static class HCDistributedMapImpl<V> implements DistributedMap<V> {
- private IMap<String, DrillSerializable> m;
- private Class<V> clazz;
+ public static class HCDistributedMapImpl<V extends DrillSerializable> implements DistributedMap<V> {
+ private IMap<String, V> m;
- public HCDistributedMapImpl(IMap m, Class<V> clazz) {
+ public HCDistributedMapImpl(IMap<String, V> m, Class<V> clazz) {
this.m = m;
- this.clazz = clazz;
}
- public DrillSerializable get(String key) {
+ public V get(String key) {
return m.get(key);
}
- public void put(String key, DrillSerializable value) {
+ public void put(String key, V value) {
m.put(key, value);
}
- public void putIfAbsent(String key, DrillSerializable value) {
+ public void putIfAbsent(String key, V value) {
m.putIfAbsent(key, value);
}
- public void putIfAbsent(String key, DrillSerializable value, long ttl, TimeUnit timeunit) {
+ public void putIfAbsent(String key, V value, long ttl, TimeUnit timeunit) {
m.putIfAbsent(key, value, ttl, timeunit);
}
}
- public static class HCDistributedMultiMapImpl<V> implements DistributedMultiMap<V> {
- private com.hazelcast.core.MultiMap<String, DrillSerializable> mmap;
- private Class<V> clazz;
+ public static class HCDistributedMultiMapImpl<V extends DrillSerializable> implements DistributedMultiMap<V> {
+ private com.hazelcast.core.MultiMap<String, V> mmap;
- public HCDistributedMultiMapImpl(com.hazelcast.core.MultiMap mmap, Class<V> clazz) {
+ public HCDistributedMultiMapImpl(com.hazelcast.core.MultiMap<String, V> mmap, Class<V> clazz) {
this.mmap = mmap;
- this.clazz = clazz;
}
- public Collection<DrillSerializable> get(String key) {
- List<DrillSerializable> list = Lists.newArrayList();
- for (DrillSerializable v : mmap.get(key)) {
+ public Collection<V> get(String key) {
+ List<V> list = Lists.newArrayList();
+ for (V v : mmap.get(key)) {
list.add(v);
}
return list;
}
@Override
- public void put(String key, DrillSerializable value) {
+ public void put(String key, V value) {
mmap.put(key, value);
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
index e6275c156..38de688e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
@@ -17,7 +17,6 @@
*/
package org.apache.drill.exec.cache;
-import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.util.Collection;
@@ -27,29 +26,28 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.io.ByteArrayDataInput;
-import com.google.common.io.ByteArrayDataOutput;
-import com.google.common.io.ByteStreams;
import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.exec.exception.DrillbitStartupException;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
-import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.server.DrillbitContext;
+import com.google.common.io.ByteArrayDataInput;
+import com.google.common.io.ByteArrayDataOutput;
+import com.google.common.io.ByteStreams;
public class LocalCache implements DistributedCache {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalCache.class);
private volatile Map<FragmentHandle, PlanFragment> handles;
- private volatile ConcurrentMap<Class, DistributedMap> maps;
- private volatile ConcurrentMap<Class, DistributedMultiMap> multiMaps;
+ private volatile ConcurrentMap<Class<?>, DistributedMap<?>> maps;
+ private volatile ConcurrentMap<Class<?>, DistributedMultiMap<?>> multiMaps;
private volatile ConcurrentMap<String, Counter> counters;
- private static final BufferAllocator allocator = BufferAllocator.getAllocator(DrillConfig.create());
+ private static final BufferAllocator allocator = new TopLevelAllocator();
@Override
public void close() throws IOException {
@@ -66,22 +64,22 @@ public class LocalCache implements DistributedCache {
@Override
public PlanFragment getFragment(FragmentHandle handle) {
- logger.debug("looking for fragment with handle: {}", handle);
+// logger.debug("looking for fragment with handle: {}", handle);
return handles.get(handle);
}
@Override
public void storeFragment(PlanFragment fragment) {
- logger.debug("Storing fragment: {}", fragment);
+// logger.debug("Storing fragment: {}", fragment);
handles.put(fragment.getHandle(), fragment);
}
@Override
public <V extends DrillSerializable> DistributedMultiMap<V> getMultiMap(Class<V> clazz) {
- DistributedMultiMap mmap = multiMaps.get(clazz);
+ DistributedMultiMap<V> mmap = (DistributedMultiMap<V>) multiMaps.get(clazz);
if (mmap == null) {
multiMaps.putIfAbsent(clazz, new LocalDistributedMultiMapImpl<V>(clazz));
- return multiMaps.get(clazz);
+ return (DistributedMultiMap<V>) multiMaps.get(clazz);
} else {
return mmap;
}
@@ -91,8 +89,8 @@ public class LocalCache implements DistributedCache {
public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
DistributedMap m = maps.get(clazz);
if (m == null) {
- maps.putIfAbsent(clazz, new LocalDistributedMapImpl(clazz));
- return maps.get(clazz);
+ maps.putIfAbsent(clazz, new LocalDistributedMapImpl<V>(clazz));
+ return (DistributedMap<V>) maps.get(clazz);
} else {
return m;
}
@@ -119,10 +117,10 @@ public class LocalCache implements DistributedCache {
return out;
}
- public static DrillSerializable deserialize(byte[] bytes, Class clazz) {
+ public static <V extends DrillSerializable> V deserialize(byte[] bytes, Class<V> clazz) {
ByteArrayDataInput in = ByteStreams.newDataInput(bytes);
try {
- DrillSerializable obj = (DrillSerializable)clazz.getConstructor(BufferAllocator.class).newInstance(allocator);
+ V obj = clazz.getConstructor(BufferAllocator.class).newInstance(allocator);
obj.read(in);
return obj;
} catch (InstantiationException | IllegalAccessException | IOException | NoSuchMethodException | InvocationTargetException e) {
@@ -130,18 +128,18 @@ public class LocalCache implements DistributedCache {
}
}
- public static class LocalDistributedMultiMapImpl<V> implements DistributedMultiMap<V> {
+ public static class LocalDistributedMultiMapImpl<V extends DrillSerializable> implements DistributedMultiMap<V> {
private ArrayListMultimap<String, ByteArrayDataOutput> mmap;
- private Class<DrillSerializable> clazz;
+ private Class<V> clazz;
- public LocalDistributedMultiMapImpl(Class clazz) {
+ public LocalDistributedMultiMapImpl(Class<V> clazz) {
mmap = ArrayListMultimap.create();
this.clazz = clazz;
}
@Override
- public Collection<DrillSerializable> get(String key) {
- List<DrillSerializable> list = Lists.newArrayList();
+ public Collection<V> get(String key) {
+ List<V> list = Lists.newArrayList();
for (ByteArrayDataOutput o : mmap.get(key)) {
list.add(deserialize(o.toByteArray(), this.clazz));
}
@@ -154,33 +152,33 @@ public class LocalCache implements DistributedCache {
}
}
- public static class LocalDistributedMapImpl<V> implements DistributedMap<V> {
+ public static class LocalDistributedMapImpl<V extends DrillSerializable> implements DistributedMap<V> {
private ConcurrentMap<String, ByteArrayDataOutput> m;
- private Class<DrillSerializable> clazz;
+ private Class<V> clazz;
- public LocalDistributedMapImpl(Class clazz) {
+ public LocalDistributedMapImpl(Class<V> clazz) {
m = Maps.newConcurrentMap();
this.clazz = clazz;
}
@Override
- public DrillSerializable get(String key) {
+ public V get(String key) {
if (m.get(key) == null) return null;
- return deserialize(m.get(key).toByteArray(), this.clazz);
+ return (V) deserialize(m.get(key).toByteArray(), this.clazz);
}
@Override
- public void put(String key, DrillSerializable value) {
+ public void put(String key, V value) {
m.put(key, serialize(value));
}
@Override
- public void putIfAbsent(String key, DrillSerializable value) {
+ public void putIfAbsent(String key, V value) {
m.putIfAbsent(key, serialize(value));
}
@Override
- public void putIfAbsent(String key, DrillSerializable value, long ttl, TimeUnit timeUnit) {
+ public void putIfAbsent(String key, V value, long ttl, TimeUnit timeUnit) {
m.putIfAbsent(key, serialize(value));
logger.warn("Expiration not implemented in local map cache");
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java
index d03eac877..53b2bfa48 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java
@@ -17,11 +17,10 @@
*/
package org.apache.drill.exec.cache;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.WorkQueueStatus;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
-import com.google.protobuf.Parser;
import com.hazelcast.core.HazelcastInstance;
public class ProtoBufImpl {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java
index 448eecde4..d55c133bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java
@@ -17,16 +17,12 @@
*/
package org.apache.drill.exec.cache;
-import java.io.DataInput;
-import java.io.DataOutput;
import java.io.IOException;
-import com.hazelcast.nio.ObjectDataInput;
-import com.hazelcast.nio.ObjectDataOutput;
-import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
-
import com.google.protobuf.MessageLite;
import com.google.protobuf.Parser;
+import com.hazelcast.nio.ObjectDataInput;
+import com.hazelcast.nio.ObjectDataOutput;
import com.hazelcast.nio.serialization.DataSerializable;
public abstract class ProtoBufWrap<T extends MessageLite> implements DataSerializable{
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 656d07f04..ff7ab02f9 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
@@ -133,6 +133,7 @@ public class VectorAccessibleSerializable implements DrillSerializable {
retain = true;
writeToStream(output);
}
+
/**
* Serializes the VectorAccessible va and writes it to an output stream
@@ -189,9 +190,10 @@ public class VectorAccessibleSerializable implements DrillSerializable {
}
}
- private void clear() {
+ public void clear() {
if (!retain) {
batch.clear();
+ if(sv2 != null) sv2.clear();
}
}
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 8ee904207..74eed9424 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
@@ -35,7 +35,7 @@ import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.coord.ClusterCoordinator;
import org.apache.drill.exec.coord.ZKClusterCoordinator;
-import org.apache.drill.exec.memory.DirectBufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.proto.UserProtos;
@@ -59,7 +59,7 @@ public class DrillClient implements Closeable{
private UserClient client;
private volatile ClusterCoordinator clusterCoordinator;
private volatile boolean connected = false;
- private final DirectBufferAllocator allocator = new DirectBufferAllocator();
+ private final TopLevelAllocator allocator = new TopLevelAllocator(Long.MAX_VALUE);
private int reconnectTimes;
private int reconnectDelay;
@@ -107,7 +107,7 @@ public class DrillClient implements Closeable{
checkState(!endpoints.isEmpty(), "No DrillbitEndpoint can be found");
// just use the first endpoint for now
DrillbitEndpoint endpoint = endpoints.iterator().next();
- this.client = new UserClient(allocator.getUnderlyingAllocator(), new NioEventLoopGroup(config.getInt(ExecConstants.CLIENT_RPC_THREADS), new NamedThreadFactory("Client-")));
+ this.client = new UserClient(allocator, new NioEventLoopGroup(config.getInt(ExecConstants.CLIENT_RPC_THREADS), new NamedThreadFactory("Client-")));
logger.debug("Connecting to server {}:{}", endpoint.getAddress(), endpoint.getUserPort());
connect(endpoint);
connected = true;
@@ -146,7 +146,7 @@ public class DrillClient implements Closeable{
}
}
- public DirectBufferAllocator getAllocator() {
+ public TopLevelAllocator getAllocator() {
return allocator;
}
@@ -218,7 +218,7 @@ public class DrillClient implements Closeable{
@Override
public void resultArrived(QueryResultBatch result) {
- logger.debug("Result arrived. Is Last Chunk: {}. Full Result: {}", result.getHeader().getIsLastChunk(), result);
+// logger.debug("Result arrived. Is Last Chunk: {}. Full Result: {}", result.getHeader().getIsLastChunk(), result);
results.add(result);
if(result.getHeader().getIsLastChunk()){
future.set(results);
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 ef0b1e18b..cfffed967 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
@@ -19,19 +19,17 @@ package org.apache.drill.exec.client;
import java.io.File;
import java.io.FileInputStream;
-import java.io.IOException;
import java.util.List;
-import org.apache.commons.lang.StringUtils;
-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.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
import org.apache.drill.exec.record.MaterializedField;
import org.apache.drill.exec.record.VectorAccessible;
import org.apache.drill.exec.record.VectorContainer;
import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
import org.apache.drill.exec.util.VectorUtil;
import com.beust.jcommander.IParameterValidator;
@@ -42,7 +40,7 @@ import com.beust.jcommander.internal.Lists;
public class DumpCat {
- private final static BufferAllocator allocator = BufferAllocator.getAllocator(DrillConfig.create());
+ private final static BufferAllocator allocator = new TopLevelAllocator();
public static void main(String args[]) throws Exception {
DumpCat dumpCat = new DumpCat();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AccountingByteBuf.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AccountingByteBuf.java
new file mode 100644
index 000000000..1013f4787
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AccountingByteBuf.java
@@ -0,0 +1,859 @@
+/**
+ * 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.ByteBufAllocator;
+import io.netty.buffer.ByteBufProcessor;
+import io.netty.buffer.DuplicatedByteBuf;
+import io.netty.buffer.PooledUnsafeDirectByteBufL;
+import io.netty.buffer.SlicedByteBuf;
+import io.netty.buffer.SwappedByteBuf;
+import io.netty.buffer.Unpooled;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.GatheringByteChannel;
+import java.nio.channels.ScatteringByteChannel;
+import java.nio.charset.Charset;
+
+public class AccountingByteBuf extends ByteBuf{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AccountingByteBuf.class);
+
+ private final PooledUnsafeDirectByteBufL b;
+ private final Accountor acct;
+ private int size;
+
+ public AccountingByteBuf(Accountor a, PooledUnsafeDirectByteBufL b) {
+ super();
+ this.b = b;
+ this.acct = a;
+ this.size = b.capacity();
+ }
+
+ @Override
+ public int refCnt() {
+ return b.refCnt();
+ }
+
+ @Override
+ public boolean release() {
+ if(b.release()){
+ acct.release(this, size);
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public boolean release(int decrement) {
+ if(b.release(decrement)){
+ acct.release(this, size);
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public int capacity() {
+ return b.capacity();
+ }
+
+ @Override
+ public ByteBuf capacity(int newCapacity) {
+ if(newCapacity < size){
+ // TODO: once DRILL-336 is merged: do trim, update size and return
+ }
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public int maxCapacity() {
+ return size;
+ }
+
+ @Override
+ public ByteBufAllocator alloc() {
+ return b.alloc();
+ }
+
+ @Override
+ public ByteOrder order() {
+ return ByteOrder.LITTLE_ENDIAN;
+ }
+
+ @Override
+ public ByteBuf order(ByteOrder endianness) {
+ if(endianness != ByteOrder.BIG_ENDIAN) throw new UnsupportedOperationException("Drill buffers only support big endian.");
+ return this;
+ }
+
+ @Override
+ public ByteBuf unwrap() {
+ return this;
+ }
+
+ @Override
+ public boolean isDirect() {
+ return true;
+ }
+
+ @Override
+ public int readerIndex() {
+ return b.readerIndex();
+ }
+
+ @Override
+ public ByteBuf readerIndex(int readerIndex) {
+ b.readerIndex(readerIndex);
+ return this;
+ }
+
+ @Override
+ public int writerIndex() {
+ return b.writerIndex();
+ }
+
+ @Override
+ public ByteBuf writerIndex(int writerIndex) {
+ b.writerIndex(writerIndex);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setIndex(int readerIndex, int writerIndex) {
+ b.setIndex(readerIndex, writerIndex);
+ return this;
+ }
+
+ @Override
+ public int readableBytes() {
+ return b.readableBytes();
+ }
+
+ @Override
+ public int writableBytes() {
+ return b.writableBytes();
+ }
+
+ @Override
+ public int maxWritableBytes() {
+ return b.maxWritableBytes();
+ }
+
+ @Override
+ public boolean isReadable() {
+ return b.isReadable();
+ }
+
+ @Override
+ public boolean isReadable(int size) {
+ return b.isReadable(size);
+ }
+
+ @Override
+ public boolean isWritable() {
+ return b.isWritable();
+ }
+
+ @Override
+ public boolean isWritable(int size) {
+ return b.isWritable(size);
+ }
+
+ @Override
+ public ByteBuf clear() {
+ b.clear();
+ return this;
+ }
+
+ @Override
+ public ByteBuf markReaderIndex() {
+ b.markReaderIndex();
+ return this;
+ }
+
+ @Override
+ public ByteBuf resetReaderIndex() {
+ b.resetReaderIndex();
+ return this;
+ }
+
+ @Override
+ public ByteBuf markWriterIndex() {
+ b.markWriterIndex();
+ return this;
+ }
+
+ @Override
+ public ByteBuf resetWriterIndex() {
+ b.resetWriterIndex();
+ return this;
+ }
+
+ @Override
+ public ByteBuf discardReadBytes() {
+ b.discardReadBytes();
+ return this;
+ }
+
+ @Override
+ public ByteBuf discardSomeReadBytes() {
+ b.discardSomeReadBytes();
+ return this;
+ }
+
+ @Override
+ public ByteBuf ensureWritable(int minWritableBytes) {
+ b.ensureWritable(minWritableBytes);
+ return this;
+ }
+
+ @Override
+ public int ensureWritable(int minWritableBytes, boolean force) {
+ return b.ensureWritable(minWritableBytes, false);
+ }
+
+ @Override
+ public boolean getBoolean(int index) {
+ return b.getBoolean(index);
+ }
+
+ @Override
+ public byte getByte(int index) {
+ return b.getByte(index);
+ }
+
+ @Override
+ public short getUnsignedByte(int index) {
+ return b.getUnsignedByte(index);
+ }
+
+ @Override
+ public short getShort(int index) {
+ return b.getShort(index);
+ }
+
+ @Override
+ public int getUnsignedShort(int index) {
+ return b.getUnsignedShort(index);
+ }
+
+ @Override
+ public int getMedium(int index) {
+ return b.getMedium(index);
+ }
+
+ @Override
+ public int getUnsignedMedium(int index) {
+ return b.getUnsignedMedium(index);
+ }
+
+ @Override
+ public int getInt(int index) {
+ return b.getInt(index);
+ }
+
+ @Override
+ public long getUnsignedInt(int index) {
+ return b.getUnsignedInt(index);
+ }
+
+ @Override
+ public long getLong(int index) {
+ return b.getLong(index);
+ }
+
+ @Override
+ public char getChar(int index) {
+ return b.getChar(index);
+ }
+
+ @Override
+ public float getFloat(int index) {
+ return b.getFloat(index);
+ }
+
+ @Override
+ public double getDouble(int index) {
+ return b.getDouble(index);
+ }
+
+ @Override
+ public ByteBuf getBytes(int index, ByteBuf dst) {
+ b.getBytes(index, dst);
+ return this;
+ }
+
+ @Override
+ public ByteBuf getBytes(int index, ByteBuf dst, int length) {
+ b.getBytes(index, dst, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf getBytes(int index, ByteBuf dst, int dstIndex, int length) {
+ b.getBytes(index, dst, dstIndex, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf getBytes(int index, byte[] dst) {
+ b.getBytes(index, dst);
+ return this;
+ }
+
+ @Override
+ public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length) {
+ b.getBytes(index, dst, dstIndex, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf getBytes(int index, ByteBuffer dst) {
+ b.getBytes(index, dst);
+ return this;
+ }
+
+ @Override
+ public ByteBuf getBytes(int index, OutputStream out, int length) throws IOException {
+ b.getBytes(index, out, length);
+ return this;
+ }
+
+ @Override
+ public int getBytes(int index, GatheringByteChannel out, int length) throws IOException {
+ return b.getBytes(index, out, length);
+ }
+
+ @Override
+ public ByteBuf setBoolean(int index, boolean value) {
+ b.setBoolean(index, value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setByte(int index, int value) {
+ b.setByte(index, value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setShort(int index, int value) {
+ b.setShort(index, value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setMedium(int index, int value) {
+ b.setMedium(index, value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setInt(int index, int value) {
+ b.setInt(index, value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setLong(int index, long value) {
+ b.setLong(index, value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setChar(int index, int value) {
+ b.setChar(index, value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setFloat(int index, float value) {
+ b.setFloat(index, value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setDouble(int index, double value) {
+ b.setDouble(index, value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setBytes(int index, ByteBuf src) {
+ b.setBytes(index, src);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setBytes(int index, ByteBuf src, int length) {
+ b.setBytes(index, src, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setBytes(int index, ByteBuf src, int srcIndex, int length) {
+ b.setBytes(index, src, srcIndex, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setBytes(int index, byte[] src) {
+ b.setBytes(index, src);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setBytes(int index, byte[] src, int srcIndex, int length) {
+ b.setBytes(index, src, srcIndex, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf setBytes(int index, ByteBuffer src) {
+ b.setBytes(index, src);
+ return this;
+ }
+
+ @Override
+ public int setBytes(int index, InputStream in, int length) throws IOException {
+ return b.setBytes(index, in, length);
+ }
+
+ @Override
+ public int setBytes(int index, ScatteringByteChannel in, int length) throws IOException {
+ return b.setBytes(index, in, length);
+ }
+
+ @Override
+ public ByteBuf setZero(int index, int length) {
+ b.setZero(index, length);
+ return this;
+ }
+
+ @Override
+ public boolean readBoolean() {
+ return b.readBoolean();
+ }
+
+ @Override
+ public byte readByte() {
+ return b.readByte();
+ }
+
+ @Override
+ public short readUnsignedByte() {
+ return b.readUnsignedByte();
+ }
+
+ @Override
+ public short readShort() {
+ return b.readShort();
+ }
+
+ @Override
+ public int readUnsignedShort() {
+ return b.readUnsignedShort();
+ }
+
+ @Override
+ public int readMedium() {
+ return b.readMedium();
+ }
+
+ @Override
+ public int readUnsignedMedium() {
+ return b.readUnsignedMedium();
+ }
+
+ @Override
+ public int readInt() {
+ return b.readInt();
+ }
+
+ @Override
+ public long readUnsignedInt() {
+ return b.readUnsignedInt();
+ }
+
+ @Override
+ public long readLong() {
+ return b.readLong();
+ }
+
+ @Override
+ public char readChar() {
+ return b.readChar();
+ }
+
+ @Override
+ public float readFloat() {
+ return b.readFloat();
+ }
+
+ @Override
+ public double readDouble() {
+ return b.readDouble();
+ }
+
+ @Override
+ public ByteBuf readBytes(int length) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ByteBuf readSlice(int length) {
+ ByteBuf slice = slice(readerIndex(), length);
+ readerIndex(readerIndex() + length);
+ return slice;
+ }
+
+
+ @Override
+ public ByteBuf readBytes(ByteBuf dst) {
+ b.readBytes(dst);
+ return this;
+ }
+
+ @Override
+ public ByteBuf readBytes(ByteBuf dst, int length) {
+ b.readBytes(dst, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf readBytes(ByteBuf dst, int dstIndex, int length) {
+ b.readBytes(dst, dstIndex, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf readBytes(byte[] dst) {
+ b.readBytes(dst);
+ return this;
+ }
+
+ @Override
+ public ByteBuf readBytes(byte[] dst, int dstIndex, int length) {
+ b.readBytes(dst, dstIndex, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf readBytes(ByteBuffer dst) {
+ b.readBytes(dst);
+ return this;
+ }
+
+ @Override
+ public ByteBuf readBytes(OutputStream out, int length) throws IOException {
+ b.readBytes(out, length);
+ return null;
+ }
+
+ @Override
+ public int readBytes(GatheringByteChannel out, int length) throws IOException {
+ return b.readBytes(out, length);
+ }
+
+ @Override
+ public ByteBuf skipBytes(int length) {
+ b.skipBytes(length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeBoolean(boolean value) {
+ b.writeBoolean(value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeByte(int value) {
+ b.writeByte(value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeShort(int value) {
+ b.writeShort(value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeMedium(int value) {
+ b.writeMedium(value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeInt(int value) {
+ b.writeInt(value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeLong(long value) {
+ b.writeLong(value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeChar(int value) {
+ b.writeChar(value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeFloat(float value) {
+ b.writeFloat(value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeDouble(double value) {
+ b.writeDouble(value);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeBytes(ByteBuf src) {
+ b.writeBytes(src);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeBytes(ByteBuf src, int length) {
+ b.writeBytes(src, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeBytes(ByteBuf src, int srcIndex, int length) {
+ b.writeBytes(src, srcIndex, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeBytes(byte[] src) {
+ b.writeBytes(src);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeBytes(byte[] src, int srcIndex, int length) {
+ b.writeBytes(src, srcIndex, length);
+ return this;
+ }
+
+ @Override
+ public ByteBuf writeBytes(ByteBuffer src) {
+ b.writeBytes(src);
+ return this;
+ }
+
+ @Override
+ public int writeBytes(InputStream in, int length) throws IOException {
+ return b.writeBytes(in, length);
+ }
+
+ @Override
+ public int writeBytes(ScatteringByteChannel in, int length) throws IOException {
+ return b.writeBytes(in, length);
+ }
+
+ @Override
+ public ByteBuf writeZero(int length) {
+ b.writeZero(length);
+ return this;
+ }
+
+ @Override
+ public int indexOf(int fromIndex, int toIndex, byte value) {
+ return b.indexOf(fromIndex, toIndex, value);
+ }
+
+ @Override
+ public int bytesBefore(byte value) {
+ return b.bytesBefore(value);
+ }
+
+ @Override
+ public int bytesBefore(int length, byte value) {
+ return b.bytesBefore(length, value);
+ }
+
+ @Override
+ public int bytesBefore(int index, int length, byte value) {
+ return b.bytesBefore(index, length, value);
+ }
+
+ @Override
+ public int forEachByte(ByteBufProcessor processor) {
+ return b.forEachByte(processor);
+ }
+
+ @Override
+ public int forEachByte(int index, int length, ByteBufProcessor processor) {
+ return b.forEachByte(index, length, processor);
+ }
+
+ @Override
+ public int forEachByteDesc(ByteBufProcessor processor) {
+ return b.forEachByteDesc(processor);
+ }
+
+ @Override
+ public int forEachByteDesc(int index, int length, ByteBufProcessor processor) {
+ return b.forEachByteDesc(index, length, processor);
+ }
+
+ @Override
+ public ByteBuf copy() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ByteBuf copy(int index, int length) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ByteBuf slice() {
+ return slice(b.readerIndex(), readableBytes());
+ }
+
+ @Override
+ public ByteBuf slice(int index, int length) {
+ if (length == 0) {
+ return Unpooled.EMPTY_BUFFER;
+ }
+
+ return new SlicedByteBuf(this, index, length);
+ }
+
+ @Override
+ public ByteBuf duplicate() {
+ return new DuplicatedByteBuf(this);
+ }
+
+ @Override
+ public int nioBufferCount() {
+ return b.nioBufferCount();
+ }
+
+ @Override
+ public ByteBuffer nioBuffer() {
+ return b.nioBuffer();
+ }
+
+ @Override
+ public ByteBuffer nioBuffer(int index, int length) {
+ return b.nioBuffer(index, length);
+ }
+
+ @Override
+ public ByteBuffer internalNioBuffer(int index, int length) {
+ return b.internalNioBuffer(index, length);
+ }
+
+ @Override
+ public ByteBuffer[] nioBuffers() {
+ return b.nioBuffers();
+ }
+
+ @Override
+ public ByteBuffer[] nioBuffers(int index, int length) {
+ return b.nioBuffers(index, length);
+ }
+
+ @Override
+ public boolean hasArray() {
+ return b.hasArray();
+ }
+
+ @Override
+ public byte[] array() {
+ return b.array();
+ }
+
+ @Override
+ public int arrayOffset() {
+ return b.arrayOffset();
+ }
+
+ @Override
+ public boolean hasMemoryAddress() {
+ return b.hasMemoryAddress();
+ }
+
+ @Override
+ public long memoryAddress() {
+ return b.memoryAddress();
+ }
+
+ @Override
+ public String toString(Charset charset) {
+ return b.toString(charset);
+ }
+
+ @Override
+ public String toString(int index, int length, Charset charset) {
+ return b.toString(index, length, charset);
+ }
+
+ @Override
+ public int hashCode() {
+ return System.identityHashCode(this);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ // identity equals only.
+ return this == obj;
+ }
+
+ @Override
+ public String toString() {
+ return "AccountingByteBuf [Inner buffer=" + b + ", size=" + size + "]";
+ }
+
+ @Override
+ public int compareTo(ByteBuf buffer) {
+ return b.compareTo(buffer);
+ }
+
+
+ @Override
+ public ByteBuf retain(int increment) {
+ b.retain(increment);
+ return this;
+ }
+
+ @Override
+ public ByteBuf retain() {
+ b.retain();
+ return this;
+ }
+
+
+}
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
new file mode 100644
index 000000000..422166427
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java
@@ -0,0 +1,190 @@
+/**
+ * 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 java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ConcurrentMap;
+
+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<ByteBuf, DebugStackTrace> buffers = Maps.newConcurrentMap();
+ private final FragmentHandle handle;
+
+ public Accountor(FragmentHandle handle, Accountor parent, long max, long preAllocated) {
+ // TODO: fix preallocation stuff
+ AtomicRemainder parentRemainder = parent != null ? parent.remainder : null;
+ this.remainder = new AtomicRemainder(parentRemainder, max, preAllocated);
+ this.total = max;
+ this.handle = handle;
+ if (ENABLE_ACCOUNTING) {
+ buffers = Maps.newConcurrentMap();
+ } else {
+ buffers = null;
+ }
+ }
+
+ public long getCapacity() {
+ return total;
+ }
+
+ public long getAllocation() {
+ return remainder.getUsed();
+ }
+
+ public boolean reserve(long size) {
+ return remainder.get(size);
+ }
+
+ public void forceAdditionalReservation(long size) {
+ remainder.forceGet(size);
+ }
+
+ public void reserved(long expected, AccountingByteBuf buf, String desc){
+ // make sure to take away the additional memory that happened due to rounding.
+
+ long additional = buf.capacity() - expected;
+ remainder.forceGet(additional);
+
+ if (ENABLE_ACCOUNTING) {
+ buffers.put(buf, new DebugStackTrace(desc, buf.capacity(), Thread.currentThread().getStackTrace()));
+ }
+ }
+ public void reserved(long expected, AccountingByteBuf buf) {
+ reserved(expected, buf, null);
+ }
+
+ public void release(AccountingByteBuf 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);
+ }
+ }
+
+ public void close() {
+ 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<DebugStackTrace, DebugStackTrace> multi = LinkedListMultimap.create();
+ for (DebugStackTrace t : buffers.values()) {
+ multi.put(t, t);
+ }
+
+ for (DebugStackTrace entry : multi.keySet()) {
+ Collection<DebugStackTrace> 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);
+ if(alloc.desc != null){
+ sb.append(" (");
+ sb.append(alloc.desc);
+ sb.append(")");
+ }
+ sb.append(", ");
+ }
+
+ sb.append("at stack location:\n");
+ entry.addToString(sb);
+ }
+
+ throw new IllegalStateException(sb.toString());
+
+ }
+
+
+ }
+
+ private class DebugStackTrace {
+
+ private StackTraceElement[] elements;
+ private long size;
+ private String desc;
+
+ public DebugStackTrace(String desc, long size, StackTraceElement[] elements) {
+ super();
+ this.elements = elements;
+ this.size = size;
+ this.desc = desc;
+ }
+
+ 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/AtomicRemainder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java
new file mode 100644
index 000000000..927e5a297
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java
@@ -0,0 +1,118 @@
+/**
+ * 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 final AtomicRemainder parent;
+ private final AtomicLong total;
+ private final AtomicLong unaccountable;
+ private final long max;
+ private final long pre;
+
+ public AtomicRemainder(AtomicRemainder parent, long max, long pre) {
+ this.parent = parent;
+ this.total = new AtomicLong(max - pre);
+ this.unaccountable = new AtomicLong(pre);
+ this.max = max;
+ this.pre = pre;
+ }
+
+ public long getRemainder() {
+ return total.get() + unaccountable.get();
+ }
+
+ public long getUsed() {
+ return max - getRemainder();
+ }
+
+ /**
+ * Automatically allocate memory. This is used when an actual allocation happened to be larger than requested. This
+ * memory has already been used up so it must be accurately accounted for in future allocations.
+ *
+ * @param size
+ */
+ public void forceGet(long size) {
+ total.addAndGet(size);
+ if (parent != null)
+ parent.forceGet(size);
+ }
+
+ public boolean get(long size) {
+ if (unaccountable.get() < 1) {
+ // if there is no preallocated memory, we can operate normally.
+ long outcome = total.addAndGet(-size);
+ if (outcome < 0) {
+ total.addAndGet(size);
+ return false;
+ } else {
+ return true;
+ }
+ } else {
+ // if there is preallocated memory, use that first.
+ long unaccount = unaccountable.getAndAdd(-size);
+ if (unaccount > -1) {
+ return true;
+ } else {
+
+ // if there is a parent allocator, check it before allocating.
+ if (parent != null && !parent.get(-unaccount)) {
+ unaccountable.getAndAdd(size);
+ return false;
+ }
+
+ long account = total.addAndGet(unaccount);
+ if (account >= 0) {
+ unaccountable.getAndAdd(unaccount);
+ return true;
+ } else {
+ unaccountable.getAndAdd(size);
+ total.addAndGet(-unaccount);
+ 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 preSize = unaccountable.get();
+ long preChange = Math.min(size, pre - preSize);
+ long totalChange = size - preChange;
+ unaccountable.addAndGet(preChange);
+ total.addAndGet(totalChange);
+ if (parent != null){
+ parent.returnAllocation(totalChange);
+ }
+ }
+
+}
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 158409216..6cb2fba38 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
@@ -17,17 +17,16 @@
*/
package org.apache.drill.exec.memory;
-import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import java.io.Closeable;
-import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
/**
* Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods. Also allows inser
*/
-public abstract class BufferAllocator implements Closeable{
+public interface BufferAllocator extends Closeable{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BufferAllocator.class);
/**
@@ -35,31 +34,23 @@ public abstract class BufferAllocator implements Closeable{
* @param size The size in bytes.
* @return A new ByteBuf.
*/
- public abstract ByteBuf buffer(int size);
+ public abstract AccountingByteBuf buffer(int size);
- public abstract ByteBufAllocator getUnderlyingAllocator();
- public abstract BufferAllocator getChildAllocator(long initialReservation, long maximumReservation);
+ public abstract AccountingByteBuf buffer(int size, String desc);
+
+ public abstract ByteBufAllocator getUnderlyingAllocator();
- protected abstract boolean pre(int bytes);
+ public abstract BufferAllocator getChildAllocator(FragmentHandle handle, long initialReservation, long maximumReservation) throws OutOfMemoryException;
- public PreAllocator getPreAllocator(){
- return new PreAllocator();
- }
+ public PreAllocator getNewPreAllocator();
- public class PreAllocator{
- int bytes = 0;
- public boolean preAllocate(int bytes){
- if(!pre(bytes)) return false;
-
- this.bytes += bytes;
- return true;
-
- }
-
- public ByteBuf getAllocation(){
- return buffer(bytes);
- }
+ /**
+ * Not thread safe.
+ */
+ public interface PreAllocator{
+ public boolean preAllocate(int bytes);
+ public AccountingByteBuf getAllocation();
}
@@ -79,11 +70,6 @@ public abstract class BufferAllocator implements Closeable{
@Override
public abstract void close();
- public static BufferAllocator getAllocator(DrillConfig config){
- // TODO: support alternative allocators (including a debugging allocator that records all allocation locations for each buffer).
- return new DirectBufferAllocator();
- }
-
public abstract long getAllocatedMemory();
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
deleted file mode 100644
index 77a251d58..000000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
+++ /dev/null
@@ -1,65 +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.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocatorL;
-
-public class DirectBufferAllocator extends BufferAllocator {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectBufferAllocator.class);
-
- private final PooledByteBufAllocatorL buffer = new PooledByteBufAllocatorL(true);
-
- public DirectBufferAllocator() {
- }
-
- @Override
- public ByteBuf buffer(int size) {
- // TODO: wrap it
- return buffer.directBuffer(size);
- }
-
- @Override
- protected boolean pre(int bytes) {
- // TODO: check allocation
- return true;
- }
-
- @Override
- public long getAllocatedMemory() {
- return 0;
- }
-
- @Override
- public ByteBufAllocator getUnderlyingAllocator() {
- return buffer;
- }
-
- @Override
- public BufferAllocator getChildAllocator(long initialReservation, long maximumReservation) {
- // TODO: Add child account allocator.
- return this;
- }
-
- @Override
- public void close() {
- // TODO: collect all buffers and release them away using a weak hashmap so we don't impact pool work
- }
-
-}
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
new file mode 100644
index 000000000..3e89ca535
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/OutOfMemoryException.java
@@ -0,0 +1,51 @@
+/**
+ * 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.exception.FragmentSetupException;
+
+public class OutOfMemoryException extends FragmentSetupException{
+ 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) {
+ 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/TopLevelAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
new file mode 100644
index 000000000..0325ce0be
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
@@ -0,0 +1,162 @@
+/**
+ * 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.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocatorL;
+import io.netty.buffer.PooledUnsafeDirectByteBufL;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+
+public class TopLevelAllocator implements BufferAllocator {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TopLevelAllocator.class);
+
+ private final PooledByteBufAllocatorL innerAllocator = new PooledByteBufAllocatorL(true);
+ private final Accountor acct;
+
+ public TopLevelAllocator() {
+ this(DrillConfig.getMaxDirectMemory());
+ }
+
+ public TopLevelAllocator(long maximumAllocation) {
+ this.acct = new Accountor(null, null, maximumAllocation, 0);
+ }
+
+ @Override
+ public AccountingByteBuf buffer(int size) {
+ return buffer(size, null);
+ }
+
+ public AccountingByteBuf buffer(int size, String desc){
+ if(!acct.reserve(size)) return null;
+ ByteBuf buffer = innerAllocator.directBuffer(size);
+ AccountingByteBuf wrapped = new AccountingByteBuf(acct, (PooledUnsafeDirectByteBufL) buffer);
+ acct.reserved(size, wrapped, desc);
+ return wrapped;
+ }
+
+ @Override
+ public long getAllocatedMemory() {
+ return acct.getAllocation();
+ }
+
+ @Override
+ public ByteBufAllocator getUnderlyingAllocator() {
+ return innerAllocator;
+ }
+
+ @Override
+ public BufferAllocator getChildAllocator(FragmentHandle handle, long initialReservation, long maximumReservation) throws OutOfMemoryException {
+ if(!acct.reserve(initialReservation)){
+ throw new OutOfMemoryException(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()));
+ };
+ return new ChildAllocator(handle, acct, initialReservation, maximumReservation);
+ }
+
+ @Override
+ public void close() {
+ acct.close();
+ }
+
+
+ private class ChildAllocator implements BufferAllocator{
+
+ private Accountor innerAcct;
+
+ public ChildAllocator(FragmentHandle handle, Accountor parentAccountor, long max, long pre) throws OutOfMemoryException{
+ innerAcct = new Accountor(handle, parentAccountor, max, pre);
+ }
+
+
+ @Override
+ public AccountingByteBuf buffer(int size, String desc) {
+ if(!innerAcct.reserve(size)){
+ return null;
+ };
+
+ ByteBuf buffer = innerAllocator.directBuffer(size);
+ AccountingByteBuf wrapped = new AccountingByteBuf(innerAcct, (PooledUnsafeDirectByteBufL) buffer);
+ innerAcct.reserved(size, wrapped);
+ return wrapped;
+ }
+
+ public AccountingByteBuf buffer(int size) {
+ return buffer(size, null);
+ }
+
+ @Override
+ public ByteBufAllocator getUnderlyingAllocator() {
+ return innerAllocator;
+ }
+
+ @Override
+ public BufferAllocator getChildAllocator(FragmentHandle handle, long initialReservation, long maximumReservation)
+ throws OutOfMemoryException {
+ return new ChildAllocator(handle, innerAcct, maximumReservation, initialReservation);
+ }
+
+ public PreAllocator getNewPreAllocator(){
+ return new PreAlloc(this.innerAcct);
+ }
+
+ @Override
+ public void close() {
+ innerAcct.close();
+ }
+
+ @Override
+ public long getAllocatedMemory() {
+ return innerAcct.getAllocation();
+ }
+
+ }
+
+ public PreAllocator getNewPreAllocator(){
+ return new PreAlloc(this.acct);
+ }
+
+ public class PreAlloc implements PreAllocator{
+ int bytes = 0;
+ final Accountor acct;
+ private PreAlloc(Accountor acct){
+ this.acct = acct;
+ }
+
+ /**
+ *
+ */
+ public boolean preAllocate(int bytes){
+
+ if(!acct.reserve(bytes)){
+ return false;
+ }
+ this.bytes += bytes;
+ return true;
+
+ }
+
+
+ public AccountingByteBuf getAllocation(){
+ AccountingByteBuf b = new AccountingByteBuf(acct, (PooledUnsafeDirectByteBufL) innerAllocator.buffer(bytes));
+ acct.reserved(bytes, b);
+ return b;
+ }
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
index 01385cf16..18a39acd4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
@@ -17,13 +17,12 @@
*/
package org.apache.drill.exec.metrics;
-import org.apache.drill.exec.server.DrillbitContext;
-
import com.codahale.metrics.Counter;
+import com.codahale.metrics.MetricRegistry;
/**
* Wraps a parent counter so that local in-thread metrics can be collected while collecting for a global counter. Note
- * that this one writer, many reader safe.
+ * that this is one writer, many reader safe.
*/
public class SingleThreadNestedCounter {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleThreadNestedCounter.class);
@@ -31,9 +30,9 @@ public class SingleThreadNestedCounter {
private volatile long count;
private final Counter counter;
- public SingleThreadNestedCounter(DrillbitContext context, String name) {
+ public SingleThreadNestedCounter(MetricRegistry registry, String name) {
super();
- this.counter = context.getMetrics().counter(name);
+ this.counter = registry.counter(name);
}
public long inc(long n) {
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 f75cf5f7c..1a1025ad1 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
@@ -17,7 +17,10 @@
*/
package org.apache.drill.exec.ops;
+import java.io.Closeable;
import java.io.IOException;
+import java.util.List;
+import java.util.Map;
import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.exec.compile.ClassTransformer;
@@ -26,56 +29,57 @@ import org.apache.drill.exec.exception.ClassTransformationException;
import org.apache.drill.exec.expr.CodeGenerator;
import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.OutOfMemoryException;
import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.control.ControlTunnel;
+import org.apache.drill.exec.rpc.data.DataTunnel;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.work.FragmentRunner;
import org.apache.drill.exec.work.batch.IncomingBuffers;
+import org.apache.drill.exec.work.fragment.FragmentExecutor;
+import com.beust.jcommander.internal.Lists;
+import com.beust.jcommander.internal.Maps;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer;
/**
- * Contextual objects required for execution of a particular fragment.
+ * Contextual objects required for execution of a particular fragment.
*/
-public class FragmentContext {
+public class FragmentContext implements Closeable {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentContext.class);
- private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentRunner.class, "completionTimes");
- private final static String METRIC_BATCHES_COMPLETED = MetricRegistry.name(FragmentRunner.class, "batchesCompleted");
- private final static String METRIC_RECORDS_COMPLETED = MetricRegistry.name(FragmentRunner.class, "recordsCompleted");
- private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentRunner.class, "dataProcessed");
+
+ private Map<DrillbitEndpoint, DataTunnel> tunnels = Maps.newHashMap();
private final DrillbitContext context;
- public final SingleThreadNestedCounter batchesCompleted;
- public final SingleThreadNestedCounter recordsCompleted;
- public final SingleThreadNestedCounter dataProcessed;
- public final Timer fragmentTime;
- private final FragmentHandle handle;
private final UserClientConnection connection;
- private IncomingBuffers buffers;
- private volatile Throwable failureCause;
- private volatile boolean failed = false;
+ private final FragmentStats stats;
private final FunctionImplementationRegistry funcRegistry;
private final QueryClassLoader loader;
private final ClassTransformer transformer;
-
- public FragmentContext(DrillbitContext dbContext, FragmentHandle handle, UserClientConnection connection, FunctionImplementationRegistry funcRegistry) {
+ private final BufferAllocator allocator;
+ private final PlanFragment fragment;
+ private List<Thread> daemonThreads = Lists.newLinkedList();
+ private IncomingBuffers buffers;
+
+ private volatile Throwable failureCause;
+ private volatile boolean failed = false;
+
+ public FragmentContext(DrillbitContext dbContext, PlanFragment fragment, UserClientConnection connection,
+ FunctionImplementationRegistry funcRegistry) throws OutOfMemoryException {
this.loader = new QueryClassLoader(true);
this.transformer = new ClassTransformer();
- this.fragmentTime = dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
- this.batchesCompleted = new SingleThreadNestedCounter(dbContext, METRIC_BATCHES_COMPLETED);
- this.recordsCompleted = new SingleThreadNestedCounter(dbContext, METRIC_RECORDS_COMPLETED);
- this.dataProcessed = new SingleThreadNestedCounter(dbContext, METRIC_DATA_PROCESSED);
+ this.stats = new FragmentStats(dbContext.getMetrics());
this.context = dbContext;
this.connection = connection;
- this.handle = handle;
+ this.fragment = fragment;
this.funcRegistry = funcRegistry;
+ this.allocator = context.getAllocator().getChildAllocator(fragment.getHandle(), fragment.getMemInitial(), fragment.getMemMax());
}
public void setBuffers(IncomingBuffers buffers) {
@@ -87,67 +91,110 @@ public class FragmentContext {
failed = true;
failureCause = cause;
}
-
- public DrillbitContext getDrillbitContext(){
+
+ public DrillbitContext getDrillbitContext() {
return context;
}
- public DrillbitEndpoint getIdentity(){
+ /**
+ * Get this node's identity.
+ * @return A DrillbitEndpoint object.
+ */
+ public DrillbitEndpoint getIdentity() {
return context.getEndpoint();
}
+
+ public FragmentStats getStats(){
+ return this.stats;
+ }
+ /**
+ * The FragmentHandle for this Fragment
+ * @return FragmentHandle
+ */
public FragmentHandle getHandle() {
- return handle;
+ return fragment.getHandle();
}
- public BufferAllocator getAllocator(){
- // TODO: A local query allocator to ensure memory limits and accurately gauge memory usage.
- return context.getAllocator();
+ /**
+ * Get this fragment's allocator.
+ * @return
+ */
+ public BufferAllocator getAllocator() {
+ return allocator;
}
- public <T> T getImplementationClass(CodeGenerator<T> cg) throws ClassTransformationException, IOException{
+ public <T> T getImplementationClass(CodeGenerator<T> cg) throws ClassTransformationException, IOException {
long t1 = System.nanoTime();
- T t= transformer.getImplementationClass(this.loader, cg.getDefinition(), cg.generate(), cg.getMaterializedClassName());
- logger.debug("Compile time: {} millis.", (System.nanoTime() - t1)/1000/1000 );
+ T t = transformer.getImplementationClass(this.loader, cg.getDefinition(), cg.generate(),
+ cg.getMaterializedClassName());
+ logger.debug("Compile time: {} millis.", (System.nanoTime() - t1) / 1000 / 1000);
return t;
-
- }
-
- public void addMetricsToStatus(FragmentStatus.Builder stats){
- stats.setBatchesCompleted(batchesCompleted.get());
- stats.setDataProcessed(dataProcessed.get());
- stats.setRecordsCompleted(recordsCompleted.get());
+
}
-
+
+ /**
+ * Get the user connection associated with this fragment. This return null unless this is a root fragment.
+ * @return The RPC connection to the query submitter.
+ */
public UserClientConnection getConnection() {
return connection;
}
- public BitCom getCommunicator(){
- return context.getBitCom();
+ public ControlTunnel getControlTunnel(DrillbitEndpoint endpoint) {
+ return context.getController().getTunnel(endpoint);
}
-
- public IncomingBuffers getBuffers(){
+
+ public DataTunnel getDataTunnel(DrillbitEndpoint endpoint, FragmentHandle remoteHandle) {
+ DataTunnel tunnel = tunnels.get(endpoint);
+ if (tunnel == null) {
+ tunnel = context.getDataConnectionsPool().getTunnel(endpoint, remoteHandle);
+ tunnels.put(endpoint, tunnel);
+ }
+ return tunnel;
+ }
+
+ /**
+ * Add a new thread to this fragment's context. This thread will likely run for the life of the fragment but should be
+ * terminated when the fragment completes. When the fragment completes, the threads will be interrupted.
+ *
+ * @param runnable
+ */
+ public void addDaemonThread(Thread thread) {
+ daemonThreads.add(thread);
+ thread.start();
+
+ }
+
+ public IncomingBuffers getBuffers() {
return buffers;
}
public Throwable getFailureCause() {
return failureCause;
}
-
- public boolean isFailed(){
+
+ public boolean isFailed() {
return failed;
}
-
- public FunctionImplementationRegistry getFunctionRegistry(){
+
+ public FunctionImplementationRegistry getFunctionRegistry() {
return funcRegistry;
}
-
- public QueryClassLoader getClassLoader(){
+
+ public QueryClassLoader getClassLoader() {
return loader;
}
public DrillConfig getConfig() {
- return context.getConfig();
+ return context.getConfig();
+ }
+
+ @Override
+ public void close() {
+ for(Thread thread: daemonThreads){
+ thread.interrupt();
+ }
+ allocator.close();
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
new file mode 100644
index 000000000..068b7fd55
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
@@ -0,0 +1,62 @@
+/**
+ * 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.ops;
+
+import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.work.fragment.FragmentExecutor;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+
+public class FragmentStats {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStats.class);
+
+ private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentExecutor.class,
+ "completionTimes");
+ private final static String METRIC_BATCHES_COMPLETED = MetricRegistry
+ .name(FragmentExecutor.class, "batchesCompleted");
+ private final static String METRIC_RECORDS_COMPLETED = MetricRegistry
+ .name(FragmentExecutor.class, "recordsCompleted");
+ private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentExecutor.class, "dataProcessed");
+
+
+
+ private final MetricRegistry metrics;
+
+ public final SingleThreadNestedCounter batchesCompleted;
+ public final SingleThreadNestedCounter recordsCompleted;
+ public final SingleThreadNestedCounter dataProcessed;
+ public final Timer fragmentTime;
+
+ public FragmentStats(MetricRegistry metrics) {
+ this.metrics = metrics;
+ this.fragmentTime = metrics.timer(METRIC_TIMER_FRAGMENT_TIME);
+ this.batchesCompleted = new SingleThreadNestedCounter(metrics, METRIC_BATCHES_COMPLETED);
+ this.recordsCompleted = new SingleThreadNestedCounter(metrics, METRIC_RECORDS_COMPLETED);
+ this.dataProcessed = new SingleThreadNestedCounter(metrics, METRIC_DATA_PROCESSED);
+ }
+
+ public void addMetricsToStatus(FragmentStatus.Builder stats) {
+ stats.setBatchesCompleted(batchesCompleted.get());
+ stats.setDataProcessed(dataProcessed.get());
+ stats.setRecordsCompleted(recordsCompleted.get());
+ }
+
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
new file mode 100644
index 000000000..ce73a0d3f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
@@ -0,0 +1,26 @@
+/**
+ * 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.ops;
+
+public class OperatorStats {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorStats.class);
+
+ private long batchesCompleted;
+ private long dataCompleted;
+
+}
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 c4cbf09b8..256e7729a 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
@@ -26,7 +26,8 @@ import org.apache.drill.exec.cache.DistributedCache;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
+import org.apache.drill.exec.rpc.data.DataConnectionCreator;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.StorageEngine;
@@ -35,11 +36,13 @@ public class QueryContext {
private QueryId queryId;
private DrillbitContext drillbitContext;
+ private WorkEventBus workBus;
public QueryContext(QueryId queryId, DrillbitContext drllbitContext) {
super();
this.queryId = queryId;
this.drillbitContext = drllbitContext;
+ this.workBus = drllbitContext.getWorkBus();
}
public DrillbitEndpoint getCurrentEndpoint(){
@@ -66,12 +69,16 @@ public class QueryContext {
return drillbitContext.getPlanReader();
}
- public BitCom getBitCom(){
- return drillbitContext.getBitCom();
+ public DataConnectionCreator getDataConnectionsPool(){
+ return drillbitContext.getDataConnectionsPool();
}
public DrillConfig getConfig(){
return drillbitContext.getConfig();
}
+ public WorkEventBus getWorkBus(){
+ return workBus;
+ }
+
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
index 48412a0d0..de5cf04ef 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
@@ -38,7 +38,7 @@ public class OrderedPartitionSender extends AbstractSender {
private final FieldReference ref;
private final List<DrillbitEndpoint> endpoints;
private final int sendingWidth;
-
+
private int recordsToSample;
private int samplingFactor;
private float completionFactor;
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 2dc5f1640..89edaf39f 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
@@ -26,6 +26,7 @@ import org.apache.drill.exec.physical.base.FragmentRoot;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.physical.impl.validate.IteratorValidatorInjector;
import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.util.AssertionUtil;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
@@ -75,11 +76,10 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
public static RootExec getExec(FragmentContext context, FragmentRoot root) throws ExecutionSetupException {
ImplCreator i = new ImplCreator();
- boolean isAssertEnabled = false;
- assert isAssertEnabled = true;
- if(isAssertEnabled){
- root = IteratorValidatorInjector.rewritePlanWithIteratorValidator(context, root);
+ if(AssertionUtil.isAssertionsEnabled()){
+ root = IteratorValidatorInjector.rewritePlanWithIteratorValidator(context, root);
}
+
root.accept(i, context);
if (i.root == null)
throw new ExecutionSetupException(
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 ce0e15f6a..24ea9c468 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
@@ -179,5 +179,9 @@ public class ScanBatch implements RecordBatch {
public WritableBatch getWritableBatch() {
return WritableBatch.get(this);
}
+
+ public void cleanup(){
+ container.clear();
+ }
}
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 d02396b10..9a6b3b1a8 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
@@ -17,7 +17,10 @@
*/
package org.apache.drill.exec.physical.impl;
-import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+
+import java.util.List;
+
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.config.Screen;
import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
@@ -31,14 +34,15 @@ import org.apache.drill.exec.record.RecordBatch.IterOutcome;
import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
-import org.apache.drill.exec.work.foreman.ErrorHelper;
+import org.apache.drill.exec.work.ErrorHelper;
-import java.util.List;
+import com.google.common.base.Preconditions;
public class ScreenCreator implements RootCreator<Screen>{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
+
@Override
public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) {
Preconditions.checkNotNull(children);
@@ -51,6 +55,8 @@ public class ScreenCreator implements RootCreator<Screen>{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenRoot.class);
volatile boolean ok = true;
+ private final SendingAccountor sendCount = new SendingAccountor();
+
final RecordBatch incoming;
final FragmentContext context;
final UserClientConnection connection;
@@ -72,7 +78,7 @@ public class ScreenCreator implements RootCreator<Screen>{
}
IterOutcome outcome = incoming.next();
- logger.debug("Screen Outcome {}", outcome);
+// logger.debug("Screen Outcome {}", outcome);
switch(outcome){
case STOP: {
QueryResult header = QueryResult.newBuilder() //
@@ -84,11 +90,12 @@ public class ScreenCreator implements RootCreator<Screen>{
.build();
QueryWritableBatch batch = new QueryWritableBatch(header);
connection.sendResult(listener, batch);
+ sendCount.increment();
return false;
}
case NONE: {
- context.batchesCompleted.inc(1);
+ context.getStats().batchesCompleted.inc(1);
QueryResult header = QueryResult.newBuilder() //
.setQueryId(context.getHandle().getQueryId()) //
.setRowCount(0) //
@@ -97,17 +104,20 @@ public class ScreenCreator implements RootCreator<Screen>{
.build();
QueryWritableBatch batch = new QueryWritableBatch(header);
connection.sendResult(listener, batch);
-
+ sendCount.increment();
+
return false;
}
case OK_NEW_SCHEMA:
materializer = new VectorRecordMaterializer(context, incoming);
// fall through.
case OK:
- context.batchesCompleted.inc(1);
- context.recordsCompleted.inc(incoming.getRecordCount());
+ context.getStats().batchesCompleted.inc(1);
+ context.getStats().recordsCompleted.inc(incoming.getRecordCount());
QueryWritableBatch batch = materializer.convertNext(false);
connection.sendResult(listener, batch);
+ sendCount.increment();
+
return true;
default:
throw new UnsupportedOperationException();
@@ -116,7 +126,8 @@ public class ScreenCreator implements RootCreator<Screen>{
@Override
public void stop() {
- incoming.kill();
+ incoming.cleanup();
+ sendCount.waitForSendComplete();
}
private SendListener listener = new SendListener();
@@ -126,7 +137,14 @@ public class ScreenCreator implements RootCreator<Screen>{
@Override
+ public void success(Ack value, ByteBuf buffer) {
+ super.success(value, buffer);
+ sendCount.decrement();
+ }
+
+ @Override
public void failed(RpcException ex) {
+ sendCount.decrement();
logger.error("Failure while sending data to user.", ex);
ErrorHelper.logAndConvertError(context.getIdentity(), "Failure while sending fragment to client.", ex, logger);
ok = false;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SendingAccountor.java
index 9307034ea..cebee5bae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SendingAccountor.java
@@ -15,39 +15,36 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.work.foreman;
+package org.apache.drill.exec.physical.impl;
-import java.util.Map;
-
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitCom;
-import org.apache.drill.exec.rpc.bit.BitTunnel;
-
-import com.google.common.collect.Maps;
+import java.util.concurrent.Semaphore;
/**
- * Keeps a local list of tunnels associated with a particular Foreman.
+ * Account for whether all messages sent have been completed. Necessary before finishing a task so we don't think
+ * buffers are hanging when they will be released.
+ *
+ * TODO: Need to update to use long for number of pending messages.
*/
-public class TunnelManager {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TunnelManager.class);
-
- private final BitCom com;
- private Map<DrillbitEndpoint, BitTunnel> tunnels = Maps.newHashMap();
-
- public TunnelManager(BitCom com){
- this.com = com;
+public class SendingAccountor {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SendingAccountor.class);
+
+ private int batchesSent = 0;
+ private Semaphore wait = new Semaphore(0);
+
+ public void increment() {
+ batchesSent++;
}
-
- public BitTunnel get(DrillbitEndpoint ep){
- BitTunnel bt = tunnels.get(ep);
- if(bt == null){
- bt = com.getTunnel(ep);
- tunnels.put(ep, bt);
+
+ public void decrement() {
+ wait.release();
+ }
+
+ public void waitForSendComplete() {
+ try {
+ wait.acquire(batchesSent);
+ batchesSent = 0;
+ } catch (InterruptedException e) {
+ logger.warn("Failure while waiting for send complete.", e);
}
-
- return bt;
}
-
-
}
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 1d1d420bc..17e233a52 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
@@ -31,7 +31,7 @@ import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.record.RecordBatch.IterOutcome;
import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.rpc.data.DataTunnel;
public class SingleSenderCreator implements RootCreator<SingleSender>{
@@ -43,21 +43,24 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
}
+
private static class SingleSenderRootExec implements RootExec{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSenderRootExec.class);
private RecordBatch incoming;
- private BitTunnel tunnel;
+ private DataTunnel tunnel;
private FragmentHandle handle;
private int recMajor;
private FragmentContext context;
private volatile boolean ok = true;
+ private final SendingAccountor sendCount = new SendingAccountor();
public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config){
this.incoming = batch;
assert(incoming != null);
this.handle = context.getHandle();
this.recMajor = config.getOppositeMajorFragmentId();
- this.tunnel = context.getCommunicator().getTunnel(config.getDestination());
+ FragmentHandle opposite = handle.toBuilder().setMajorFragmentId(config.getOppositeMajorFragmentId()).setMinorFragmentId(0).build();
+ this.tunnel = context.getDataTunnel(config.getDestination(), opposite);
this.context = context;
}
@@ -69,18 +72,20 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
return false;
}
IterOutcome out = incoming.next();
- logger.debug("Outcome of sender next {}", out);
+// logger.debug("Outcome of sender next {}", out);
switch(out){
case STOP:
case NONE:
FragmentWritableBatch b2 = FragmentWritableBatch.getEmptyLast(handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0);
- tunnel.sendRecordBatch(new RecordSendFailure(), context, b2);
+ sendCount.increment();
+ tunnel.sendRecordBatch(new RecordSendFailure(), b2);
return false;
case OK_NEW_SCHEMA:
case OK:
FragmentWritableBatch batch = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
- tunnel.sendRecordBatch(new RecordSendFailure(), context, batch);
+ sendCount.increment();
+ tunnel.sendRecordBatch(new RecordSendFailure(), batch);
return true;
case NOT_YET:
@@ -92,6 +97,8 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
@Override
public void stop() {
ok = false;
+ incoming.cleanup();
+ sendCount.waitForSendComplete();
}
@@ -99,12 +106,14 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
@Override
public void failed(RpcException ex) {
+ sendCount.decrement();
context.fail(ex);
stop();
}
@Override
public void success(Ack value, ByteBuf buf) {
+ sendCount.decrement();
if(value.getOk()) return;
logger.error("Downstream fragment was not accepted. Stopping future sends.");
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index 6a1636720..6c6e92c91 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -105,10 +105,13 @@ public class WireRecordBatch implements RecordBatch{
batch = fragProvider.getNext();
}
- if (batch == null) return IterOutcome.NONE;
+ if (batch == null){
+ batchLoader.clear();
+ return IterOutcome.NONE;
+ }
- logger.debug("Next received batch {}", batch);
+// logger.debug("Next received batch {}", batch);
RecordBatchDef rbd = batch.getHeader().getDef();
boolean schemaChanged = batchLoader.load(rbd, batch.getBody());
@@ -130,6 +133,10 @@ public class WireRecordBatch implements RecordBatch{
public WritableBatch getWritableBatch() {
return batchLoader.getWritableBatch();
}
+
+ @Override
+ public void cleanup() {
+ }
}
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 65cadb53c..4b9e3adda 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
@@ -17,21 +17,22 @@
******************************************************************************/
package org.apache.drill.exec.physical.impl.broadcastsender;
+import java.util.List;
+
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.config.BroadcastSender;
import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.physical.impl.SendingAccountor;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.GeneralRPCProtos;
import org.apache.drill.exec.record.FragmentWritableBatch;
import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.record.WritableBatch;
import org.apache.drill.exec.rpc.DrillRpcFuture;
import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitTunnel;
-
-import java.util.List;
-
-import static org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.data.DataTunnel;
/**
* Broadcast Sender broadcasts incoming batches to all receivers (one or more).
@@ -42,7 +43,7 @@ public class BroadcastSenderRootExec implements RootExec {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BroadcastSenderRootExec.class);
private final FragmentContext context;
private final BroadcastSender config;
- private final BitTunnel[] tunnels;
+ private final DataTunnel[] tunnels;
private final ExecProtos.FragmentHandle handle;
private volatile boolean ok;
private final RecordBatch incoming;
@@ -57,9 +58,10 @@ public class BroadcastSenderRootExec implements RootExec {
this.config = config;
this.handle = context.getHandle();
List<DrillbitEndpoint> destinations = config.getDestinations();
- this.tunnels = new BitTunnel[destinations.size()];
+ this.tunnels = new DataTunnel[destinations.size()];
for(int i = 0; i < destinations.size(); ++i) {
- tunnels[i] = context.getCommunicator().getTunnel(destinations.get(i));
+ FragmentHandle opp = handle.toBuilder().setMajorFragmentId(config.getOppositeMajorFragmentId()).setMinorFragmentId(i).build();
+ tunnels[i] = context.getDataTunnel(destinations.get(i), opp);
}
responseFutures = new DrillRpcFuture[destinations.size()];
}
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 d1f2e6841..73cf0c8a0 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
@@ -82,6 +82,13 @@ public class FilterRecordBatch extends AbstractSingleRecordBatch<Filter>{
}
}
+
+ @Override
+ public void cleanup() {
+ super.cleanup();
+ if(sv2 != null) sv2.clear();
+ }
+
@Override
protected void setupNewSchema() throws SchemaChangeException {
container.clear();
@@ -97,7 +104,7 @@ public class FilterRecordBatch extends AbstractSingleRecordBatch<Filter>{
break;
case FOUR_BYTE:
// set up the multi-batch selection vector
- this.svAllocator = context.getAllocator().getPreAllocator();
+ this.svAllocator = context.getAllocator().getNewPreAllocator();
if (!svAllocator.preAllocate(incoming.getRecordCount()*4))
throw new SchemaChangeException("Attempted to filter an SV4 which exceeds allowed memory (" +
incoming.getRecordCount() * 4 + " bytes)");
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 b07711ec1..f665c1f61 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
@@ -41,7 +41,7 @@ public class MergeJoinBatchBuilder {
public MergeJoinBatchBuilder(FragmentContext context, JoinStatus status) {
this.container = new VectorContainer();
this.status = status;
- this.svAllocator = context.getAllocator().getPreAllocator();
+ this.svAllocator = context.getAllocator().getNewPreAllocator();
}
public boolean add(RecordBatch batch) {
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 712af9ffb..bf2bcf8b6 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
@@ -158,7 +158,7 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
}
@Override
- protected void cleanup(){
+ public void cleanup(){
super.cleanup();
outgoingSv.clear();
}
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 f1bae1502..bd8692358 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
@@ -314,6 +314,7 @@ public class MergingRecordBatch implements RecordBatch {
for (RawFragmentBatchProvider provider : fragProviders) {
provider.kill(context);
}
+
}
@Override
@@ -631,4 +632,12 @@ public class MergingRecordBatch implements RecordBatch {
}
}
+ @Override
+ public void cleanup() {
+ outgoingContainer.clear();
+ for(RecordBatchLoader rbl : batchLoaders){
+ rbl.clear();
+ }
+ }
+
} \ No newline at end of file
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 f120233d4..e93176b6b 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
@@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.sun.codemodel.JConditional;
import com.sun.codemodel.JExpr;
+
import org.apache.drill.common.defs.OrderDef;
import org.apache.drill.common.expression.*;
import org.apache.drill.common.logical.data.Order;
@@ -50,43 +51,48 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
/**
- * The purpose of this operator is to generate an ordered partition, rather than a random hash partition. This could be used
- * to do a total order sort, for example.
- * This operator reads in a few incoming record batches, samples these batches, and stores them in the distributed cache. The samples
- * from all the parallel-running fragments are merged, and a partition-table is built and stored in the distributed cache for use by all
- * fragments. A new column is added to the outgoing batch, whose value is determined by where each record falls in the partition table.
- * This column is used by PartitionSenderRootExec to determine which bucket to assign each record to.
+ * The purpose of this operator is to generate an ordered partition, rather than a random hash partition. This could be
+ * used to do a total order sort, for example. This operator reads in a few incoming record batches, samples these
+ * batches, and stores them in the distributed cache. The samples from all the parallel-running fragments are merged,
+ * and a partition-table is built and stored in the distributed cache for use by all fragments. A new column is added to
+ * the outgoing batch, whose value is determined by where each record falls in the partition table. This column is used
+ * by PartitionSenderRootExec to determine which bucket to assign each record to.
*/
-public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPartitionSender>{
+public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPartitionSender> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OrderedPartitionRecordBatch.class);
- public final MappingSet mainMapping = new MappingSet( (String) null, null, CodeGenerator.DEFAULT_SCALAR_MAP, CodeGenerator.DEFAULT_SCALAR_MAP);
- public final MappingSet incomingMapping = new MappingSet("inIndex", null, "incoming", null, CodeGenerator.DEFAULT_SCALAR_MAP, CodeGenerator.DEFAULT_SCALAR_MAP);
+ public final MappingSet mainMapping = new MappingSet((String) null, null, CodeGenerator.DEFAULT_SCALAR_MAP,
+ CodeGenerator.DEFAULT_SCALAR_MAP);
+ public final MappingSet incomingMapping = new MappingSet("inIndex", null, "incoming", null,
+ CodeGenerator.DEFAULT_SCALAR_MAP, CodeGenerator.DEFAULT_SCALAR_MAP);
public final MappingSet partitionMapping = new MappingSet("partitionIndex", null, "partitionVectors", null,
- CodeGenerator.DEFAULT_SCALAR_MAP, CodeGenerator.DEFAULT_SCALAR_MAP);
+ CodeGenerator.DEFAULT_SCALAR_MAP, CodeGenerator.DEFAULT_SCALAR_MAP);
private static long MAX_SORT_BYTES = 8l * 1024 * 1024 * 1024;
private final int recordsToSample; // How many records must be received before analyzing
private final int samplingFactor; // Will collect samplingFactor * number of partitions to send to distributed cache
- private final float completionFactor; // What fraction of fragments must be completed before attempting to build partition table
+ private final float completionFactor; // What fraction of fragments must be completed before attempting to build
+ // partition table
protected final RecordBatch incoming;
private boolean first = true;
private OrderedPartitionProjector projector;
- private List<ValueVector> allocationVectors;
private VectorContainer partitionVectors = new VectorContainer();
private int partitions;
private Queue<VectorContainer> batchQueue;
- private SortRecordBatchBuilder builder;
private int recordsSampled;
private int sendingMajorFragmentWidth;
private boolean startedUnsampledBatches = false;
private boolean upstreamNone = false;
private int recordCount;
- private DistributedMap<VectorAccessibleSerializable> tableMap;
- private DistributedMultiMap<?> mmap;
- private String mapKey;
- public OrderedPartitionRecordBatch(OrderedPartitionSender pop, RecordBatch incoming, FragmentContext context){
+ private final IntVector partitionKeyVector;
+ private final DistributedMap<VectorAccessibleSerializable> tableMap;
+ private final Counter minorFragmentSampleCount;
+ private final DistributedMultiMap<VectorAccessibleSerializable> mmap;
+ private final String mapKey;
+ private List<VectorContainer> sampledIncomingBatches;
+
+ public OrderedPartitionRecordBatch(OrderedPartitionSender pop, RecordBatch incoming, FragmentContext context) {
super(pop, context);
this.incoming = incoming;
this.partitions = pop.getDestinations().size();
@@ -94,113 +100,156 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
this.recordsToSample = pop.getRecordsToSample();
this.samplingFactor = pop.getSamplingFactor();
this.completionFactor = pop.getCompletionFactor();
+
+ DistributedCache cache = context.getDrillbitContext().getCache();
+ this.mmap = cache.getMultiMap(VectorAccessibleSerializable.class);
+ this.tableMap = cache.getMap(VectorAccessibleSerializable.class);
+ Preconditions.checkNotNull(tableMap);
+
+ this.mapKey = String.format("%s_%d", context.getHandle().getQueryId(), context.getHandle().getMajorFragmentId());
+ this.minorFragmentSampleCount = cache.getCounter(mapKey);
+
+ SchemaPath outputPath = new SchemaPath(popConfig.getRef().getPath(), ExpressionPosition.UNKNOWN);
+ MaterializedField outputField = MaterializedField.create(outputPath, Types.required(TypeProtos.MinorType.INT));
+ this.partitionKeyVector = (IntVector) TypeHelper.getNewVector(outputField, context.getAllocator());
+
}
- /**
- * This method is called when the first batch comes in. Incoming batches are collected until a threshold is met. At that point,
- * the records in the batches are sorted and sampled, and the sampled records are stored in the distributed cache. Once a sufficient
- * fraction of the fragments have shared their samples, each fragment grabs all the samples, sorts all the records, builds a partition
- * table, and attempts to push the partition table to the distributed cache. Whichever table gets pushed first becomes the table used by all
- * fragments for partitioning.
- * @return
- */
- private boolean getPartitionVectors() {
- VectorContainer sampleContainer = new VectorContainer();
+
+ @Override
+ public void cleanup() {
+ super.cleanup();
+ this.partitionVectors.clear();
+ this.partitionKeyVector.clear();
+ }
+
+
+ private boolean saveSamples() throws SchemaChangeException, ClassTransformationException, IOException {
recordsSampled = 0;
IterOutcome upstream;
// Start collecting batches until recordsToSample records have been collected
-
- builder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES, sampleContainer);
+
+ SortRecordBatchBuilder builder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES);
builder.add(incoming);
+
recordsSampled += incoming.getRecordCount();
- try {
- outer: while (recordsSampled < recordsToSample) {
- upstream = incoming.next();
- switch(upstream) {
- case NONE:
- case NOT_YET:
- case STOP:
- upstreamNone = true;
- break outer;
- default:
- // fall through
- }
- builder.add(incoming);
- recordsSampled += incoming.getRecordCount();
- if (upstream == IterOutcome.NONE) break;
+
+ outer: while (recordsSampled < recordsToSample) {
+ upstream = incoming.next();
+ switch (upstream) {
+ case NONE:
+ case NOT_YET:
+ case STOP:
+ upstreamNone = true;
+ break outer;
+ default:
+ // fall through
}
- builder.build(context);
+ builder.add(incoming);
+ recordsSampled += incoming.getRecordCount();
+ if (upstream == IterOutcome.NONE)
+ break;
+ }
+ VectorContainer sortedSamples = new VectorContainer();
+ builder.build(context, sortedSamples);
- // Sort the records according the orderings given in the configuration
+ // Sort the records according the orderings given in the configuration
- Sorter sorter = SortBatch.createNewSorter(context, popConfig.getOrderings(), sampleContainer);
- SelectionVector4 sv4 = builder.getSv4();
- sorter.setup(context, sv4, sampleContainer);
- sorter.sort(sv4, sampleContainer);
+ Sorter sorter = SortBatch.createNewSorter(context, popConfig.getOrderings(), sortedSamples);
+ 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). Uses the
- // the expressions from the OrderDefs to populate each column. There is one column for each OrderDef in popConfig.orderings.
+ // Project every Nth record to a new vector container, where N = recordsSampled/(samplingFactor * partitions).
+ // Uses the
+ // the expressions from the OrderDefs to populate each column. There is one column for each OrderDef in
+ // popConfig.orderings.
- VectorContainer containerToCache = new VectorContainer();
- SampleCopier copier = getCopier(sv4, sampleContainer, containerToCache, popConfig.getOrderings());
- copier.copyRecords(recordsSampled/(samplingFactor * partitions), 0, samplingFactor * partitions);
+ VectorContainer containerToCache = new VectorContainer();
+ SampleCopier copier = getCopier(sv4, sortedSamples, containerToCache, popConfig.getOrderings());
+ copier.copyRecords(recordsSampled / (samplingFactor * partitions), 0, samplingFactor * partitions);
- for (VectorWrapper<?> vw : containerToCache) {
- vw.getValueVector().getMutator().setValueCount(copier.getOutputRecords());
- }
- containerToCache.setRecordCount(copier.getOutputRecords());
+ for (VectorWrapper<?> vw : containerToCache) {
+ vw.getValueVector().getMutator().setValueCount(copier.getOutputRecords());
+ }
+ containerToCache.setRecordCount(copier.getOutputRecords());
- // Get a distributed multimap handle from the distributed cache, and put the vectors from the new vector container
- // into a serializable wrapper object, and then add to distributed map
+ // Get a distributed multimap handle from the distributed cache, and put the vectors from the new vector container
+ // into a serializable wrapper object, and then add to distributed map
- DistributedCache cache = context.getDrillbitContext().getCache();
- mapKey = String.format("%s_%d", context.getHandle().getQueryId(), context.getHandle().getMajorFragmentId());
- mmap = cache.getMultiMap(VectorAccessibleSerializable.class);
- List<ValueVector> vectorList = Lists.newArrayList();
- for (VectorWrapper<?> vw : containerToCache) {
- vectorList.add(vw.getValueVector());
- }
+ WritableBatch batch = WritableBatch.getBatchNoHVWrap(containerToCache.getRecordCount(), containerToCache, false);
+ VectorAccessibleSerializable sampleToSave = new VectorAccessibleSerializable(batch, context.getAllocator());
- WritableBatch batch = WritableBatch.getBatchNoHVWrap(containerToCache.getRecordCount(), containerToCache, false);
- VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, context.getDrillbitContext().getAllocator());
+ mmap.put(mapKey, sampleToSave);
+ this.sampledIncomingBatches = builder.getHeldRecordBatches();
+ builder.clear();
+ batch.clear();
+ containerToCache.clear();
+ sampleToSave.clear();
+ return true;
+
+
+ }
+
+ /**
+ * This method is called when the first batch comes in. Incoming batches are collected until a threshold is met. At
+ * that point, the records in the batches are sorted and sampled, and the sampled records are stored in the
+ * distributed cache. Once a sufficient fraction of the fragments have shared their samples, each fragment grabs all
+ * the samples, sorts all the records, builds a partition table, and attempts to push the partition table to the
+ * distributed cache. Whichever table gets pushed first becomes the table used by all fragments for partitioning.
+ *
+ * @return True is successful. False if failed.
+ */
+ private boolean getPartitionVectors() {
- mmap.put(mapKey, wrap);
- wrap = null;
- Counter minorFragmentSampleCount = cache.getCounter(mapKey);
+ try {
+
+ if (!saveSamples()){
+ return false;
+ }
+
+ VectorAccessibleSerializable finalTable = null;
long val = minorFragmentSampleCount.incrementAndGet();
logger.debug("Incremented mfsc, got {}", val);
- tableMap = cache.getMap(VectorAccessibleSerializable.class);
- Preconditions.checkNotNull(tableMap);
- if (val == Math.ceil(sendingMajorFragmentWidth * completionFactor)) {
+ final long fragmentsBeforeProceed = (long) Math.ceil(sendingMajorFragmentWidth * completionFactor);
+ final String finalTableKey = mapKey + "final";
+
+ if (val == fragmentsBeforeProceed) { // we crossed the barrier, build table and get data.
buildTable();
- wrap = (VectorAccessibleSerializable)tableMap.get(mapKey + "final");
- } else if (val < Math.ceil(sendingMajorFragmentWidth * completionFactor)) {
- // Wait until sufficient number of fragments have submitted samples, or proceed after 100 ms passed
- for (int i = 0; i < 100 && wrap == null; i++) {
+ finalTable = tableMap.get(finalTableKey);
+ } else {
+ // Wait until sufficient number of fragments have submitted samples, or proceed after xx ms passed
+ // TODO: this should be polling.
+
+ if (val < fragmentsBeforeProceed)
Thread.sleep(10);
- wrap = (VectorAccessibleSerializable)tableMap.get(mapKey + "final");
- if (i == 99) {
- buildTable();
- wrap = (VectorAccessibleSerializable)tableMap.get(mapKey + "final");
+ for (int i = 0; i < 100 && finalTable == null; i++) {
+ finalTable = tableMap.get(finalTableKey);
+ if (finalTable != null){
+ break;
}
+ Thread.sleep(10);
}
- } else {
- wrap = (VectorAccessibleSerializable)tableMap.get(mapKey + "final");
+ if (finalTable == null){
+ buildTable();
+ }
+ finalTable = tableMap.get(finalTableKey);
}
- Preconditions.checkState(wrap != null);
+ Preconditions.checkState(finalTable != null);
- // Extract vectors from the wrapper, and add to partition vectors. These vectors will be used for partitioning in the rest of this operator
- for (VectorWrapper<?> w : wrap.get()) {
+ // Extract vectors from the wrapper, and add to partition vectors. These vectors will be used for partitioning in
+ // the rest of this operator
+ for (VectorWrapper<?> w : finalTable.get()) {
partitionVectors.add(w.getValueVector());
}
} catch (ClassTransformationException | IOException | SchemaChangeException | InterruptedException ex) {
kill();
- logger.error("Failure during query", ex);
+ logger.error("Failure while building final partition table.", ex);
context.fail(ex);
return false;
}
@@ -208,15 +257,15 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
}
private void buildTable() throws SchemaChangeException, ClassTransformationException, IOException {
- // Get all samples from distributed map
- Collection<DrillSerializable> allSamplesWrap = mmap.get(mapKey);
- VectorContainer allSamplesContainer = new VectorContainer();
- SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES, allSamplesContainer);
- for (DrillSerializable w : allSamplesWrap) {
- containerBuilder.add(((VectorAccessibleSerializable)w).get());
+ // Get all samples from distributed map
+
+ SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES);
+ for (VectorAccessibleSerializable w : mmap.get(mapKey)) {
+ containerBuilder.add(w.get());
}
- containerBuilder.build(context);
+ VectorContainer allSamplesContainer = new VectorContainer();
+ containerBuilder.build(context, allSamplesContainer);
List<OrderDef> orderDefs = Lists.newArrayList();
int i = 0;
@@ -225,33 +274,40 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
orderDefs.add(new OrderDef(od.getDirection(), new FieldReference(sp)));
}
+ // sort the data incoming samples.
SelectionVector4 newSv4 = containerBuilder.getSv4();
- Sorter sorter2 = SortBatch.createNewSorter(context, orderDefs, allSamplesContainer);
- sorter2.setup(context, newSv4, allSamplesContainer);
- sorter2.sort(newSv4, allSamplesContainer);
+ 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.
VectorContainer candidatePartitionTable = new VectorContainer();
- SampleCopier copier2 = getCopier(newSv4, allSamplesContainer, candidatePartitionTable, orderDefs);
+ SampleCopier copier = getCopier(newSv4, allSamplesContainer, candidatePartitionTable, orderDefs);
int skipRecords = containerBuilder.getSv4().getTotalCount() / partitions;
- copier2.copyRecords(skipRecords, skipRecords, partitions - 1);
- assert copier2.getOutputRecords() == partitions - 1 : String.format("output records: %d partitions: %d", copier2.getOutputRecords(), partitions);
+ 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) {
- vw.getValueVector().getMutator().setValueCount(copier2.getOutputRecords());
+ vw.getValueVector().getMutator().setValueCount(copier.getOutputRecords());
}
- candidatePartitionTable.setRecordCount(copier2.getOutputRecords());
+ candidatePartitionTable.setRecordCount(copier.getOutputRecords());
WritableBatch batch = WritableBatch.getBatchNoHVWrap(candidatePartitionTable.getRecordCount(), candidatePartitionTable, false);
-
VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, context.getDrillbitContext().getAllocator());
-
tableMap.putIfAbsent(mapKey + "final", wrap, 1, TimeUnit.MINUTES);
+
+ candidatePartitionTable.clear();
+ allSamplesContainer.clear();
+ containerBuilder.clear();
+ wrap.clear();
+
}
/**
- * Creates a copier that does a project for every Nth record from a VectorContainer incoming into VectorContainer outgoing. Each OrderDef in orderings
- * generates a column, and evaluation of the expression associated with each OrderDef determines the value of each column. These records will later be
- * sorted based on the values in each column, in the same order as the orderings.
+ * Creates a copier that does a project for every Nth record from a VectorContainer incoming into VectorContainer
+ * outgoing. Each OrderDef in orderings generates a column, and evaluation of the expression associated with each
+ * OrderDef determines the value of each column. These records will later be sorted based on the values in each
+ * column, in the same order as the orderings.
+ *
* @param sv4
* @param incoming
* @param outgoing
@@ -259,20 +315,24 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
* @return
* @throws SchemaChangeException
*/
- private SampleCopier getCopier(SelectionVector4 sv4, VectorContainer incoming, VectorContainer outgoing, List<OrderDef> orderings) throws SchemaChangeException{
+ private SampleCopier getCopier(SelectionVector4 sv4, VectorContainer incoming, VectorContainer outgoing,
+ List<OrderDef> orderings) throws SchemaChangeException {
List<ValueVector> localAllocationVectors = Lists.newArrayList();
final ErrorCollector collector = new ErrorCollectorImpl();
- final CodeGenerator<SampleCopier> cg = new CodeGenerator<SampleCopier>(SampleCopier.TEMPLATE_DEFINITION, context.getFunctionRegistry());
+ final CodeGenerator<SampleCopier> cg = new CodeGenerator<SampleCopier>(SampleCopier.TEMPLATE_DEFINITION,
+ context.getFunctionRegistry());
int i = 0;
- for(OrderDef od : orderings) {
+ for (OrderDef od : orderings) {
final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), incoming, collector, context.getFunctionRegistry());
SchemaPath schemaPath = new SchemaPath("f" + i++, ExpressionPosition.UNKNOWN);
- TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder().mergeFrom(expr.getMajorType()).clearMode().setMode(TypeProtos.DataMode.REQUIRED);
+ TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder().mergeFrom(expr.getMajorType())
+ .clearMode().setMode(TypeProtos.DataMode.REQUIRED);
TypeProtos.MajorType newType = builder.build();
MaterializedField outputField = MaterializedField.create(schemaPath, newType);
- if(collector.hasErrors()){
- throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema. Errors:\n %s.", collector.toErrorString()));
+ if (collector.hasErrors()) {
+ throw new SchemaChangeException(String.format(
+ "Failure while trying to materialize incoming schema. Errors:\n %s.", collector.toErrorString()));
}
ValueVector vector = TypeHelper.getNewVector(outputField, context.getAllocator());
@@ -302,89 +362,100 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
@Override
public IterOutcome next() {
+ container.zeroVectors();
- //if we got IterOutcome.NONE while getting partition vectors, and there are no batches on the queue, then we are done
- if (upstreamNone && (batchQueue == null || batchQueue.size() == 0)) return IterOutcome.NONE;
+ // if we got IterOutcome.NONE while getting partition vectors, and there are no batches on the queue, then we are
+ // done
+ if (upstreamNone && (batchQueue == null || batchQueue.size() == 0))
+ return IterOutcome.NONE;
// if there are batches on the queue, process them first, rather than calling incoming.next()
if (batchQueue != null && batchQueue.size() > 0) {
VectorContainer vc = batchQueue.poll();
recordCount = vc.getRecordCount();
- try{
+ try {
// Must set up a new schema each time, because ValueVectors are not reused between containers in queue
setupNewSchema(vc);
- }catch(SchemaChangeException ex){
+ } catch (SchemaChangeException ex) {
kill();
logger.error("Failure during query", ex);
context.fail(ex);
return IterOutcome.STOP;
}
doWork(vc);
+ vc.zeroVectors();
return IterOutcome.OK_NEW_SCHEMA;
}
- // Reaching this point, either this is the first iteration, or there are no batches left on the queue and there are more incoming
+ // Reaching this point, either this is the first iteration, or there are no batches left on the queue and there are
+ // more incoming
IterOutcome upstream = incoming.next();
-
- if(this.first && upstream == IterOutcome.OK) {
+ if (this.first && upstream == IterOutcome.OK) {
throw new RuntimeException("Invalid state: First batch should have OK_NEW_SCHEMA");
}
// 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()) return IterOutcome.STOP;
- batchQueue = new LinkedBlockingQueue<>(builder.getContainers());
+ if (this.first && upstream == IterOutcome.OK_NEW_SCHEMA) {
+ if (!getPartitionVectors()){
+ cleanup();
+ return IterOutcome.STOP;
+ }
+
+ batchQueue = new LinkedBlockingQueue<>(this.sampledIncomingBatches);
first = false;
// Now that we have the partition vectors, we immediately process the first batch on the queue
VectorContainer vc = batchQueue.poll();
- try{
+ try {
setupNewSchema(vc);
- }catch(SchemaChangeException ex){
+ } catch (SchemaChangeException ex) {
kill();
logger.error("Failure during query", ex);
context.fail(ex);
return IterOutcome.STOP;
}
doWork(vc);
+ vc.zeroVectors();
recordCount = vc.getRecordCount();
return IterOutcome.OK_NEW_SCHEMA;
}
- // if this now that all the batches on the queue are processed, we begin processing the incoming batches. For the first one
+ // if this now that all the batches on the queue are processed, we begin processing the incoming batches. For the
+ // first one
// we need to generate a new schema, even if the outcome is IterOutcome.OK After that we can reuse the schema.
if (this.startedUnsampledBatches == false) {
this.startedUnsampledBatches = true;
- if (upstream == IterOutcome.OK) upstream = IterOutcome.OK_NEW_SCHEMA;
+ if (upstream == IterOutcome.OK)
+ upstream = IterOutcome.OK_NEW_SCHEMA;
}
- switch(upstream){
- case NONE:
- case NOT_YET:
- case STOP:
- container.zeroVectors();
- recordCount = 0;
- return upstream;
- case OK_NEW_SCHEMA:
- try{
- setupNewSchema(incoming);
- }catch(SchemaChangeException ex){
- kill();
- logger.error("Failure during query", ex);
- context.fail(ex);
- return IterOutcome.STOP;
- }
- // fall through.
- case OK:
- doWork(incoming);
- recordCount = incoming.getRecordCount();
- return upstream; // change if upstream changed, otherwise normal.
- default:
- throw new UnsupportedOperationException();
+ switch (upstream) {
+ case NONE:
+ case NOT_YET:
+ case STOP:
+ cleanup();
+ recordCount = 0;
+ return upstream;
+ case OK_NEW_SCHEMA:
+ try {
+ setupNewSchema(incoming);
+ } catch (SchemaChangeException ex) {
+ kill();
+ logger.error("Failure during query", ex);
+ context.fail(ex);
+ return IterOutcome.STOP;
+ }
+ // fall through.
+ case OK:
+ doWork(incoming);
+ recordCount = incoming.getRecordCount();
+ return upstream; // change if upstream changed, otherwise normal.
+ default:
+ throw new UnsupportedOperationException();
}
}
-
+
@Override
public int getRecordCount() {
return recordCount;
@@ -392,31 +463,30 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
protected void doWork(VectorAccessible batch) {
int recordCount = batch.getRecordCount();
- for(ValueVector v : this.allocationVectors){
- AllocationHelper.allocate(v, recordCount, 50);
- }
+ AllocationHelper.allocate(partitionKeyVector, recordCount, 50);
projector.projectRecords(recordCount, 0);
- for(VectorWrapper<?> v : container){
+ for (VectorWrapper<?> v : container) {
ValueVector.Mutator m = v.getValueVector().getMutator();
m.setValueCount(recordCount);
}
}
/**
- * Sets up projection that will transfer all of the columns in batch, and also populate the partition column based on which
- * partition a record falls into in the partition table
+ * Sets up projection that will transfer all of the columns in batch, and also populate the partition column based on
+ * which partition a record falls into in the partition table
+ *
* @param batch
* @throws SchemaChangeException
*/
- protected void setupNewSchema(VectorAccessible batch) throws SchemaChangeException{
- this.allocationVectors = Lists.newArrayList();
+ protected void setupNewSchema(VectorAccessible batch) throws SchemaChangeException {
container.clear();
final ErrorCollector collector = new ErrorCollectorImpl();
final List<TransferPair> transfers = Lists.newArrayList();
-
- final CodeGenerator<OrderedPartitionProjector> cg = new CodeGenerator<OrderedPartitionProjector>(OrderedPartitionProjector.TEMPLATE_DEFINITION, context.getFunctionRegistry());
- for (VectorWrapper vw : batch) {
+ final CodeGenerator<OrderedPartitionProjector> cg = new CodeGenerator<OrderedPartitionProjector>(
+ OrderedPartitionProjector.TEMPLATE_DEFINITION, context.getFunctionRegistry());
+
+ for (VectorWrapper<?> vw : batch) {
TransferPair tp = vw.getValueVector().getTransferPair();
transfers.add(tp);
container.add(tp.getTo());
@@ -425,42 +495,41 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
cg.setMappingSet(mainMapping);
int count = 0;
- for(OrderDef od : popConfig.getOrderings()){
+ for (OrderDef od : popConfig.getOrderings()) {
final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), batch, collector, context.getFunctionRegistry());
- if(collector.hasErrors()) throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
+ if (collector.hasErrors())
+ throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
cg.setMappingSet(incomingMapping);
CodeGenerator.HoldingContainer left = cg.addExpr(expr, false);
cg.setMappingSet(partitionMapping);
- CodeGenerator.HoldingContainer right = cg.addExpr(new ValueVectorReadExpression(new TypedFieldId(expr.getMajorType(), count++)), false);
+ CodeGenerator.HoldingContainer right = cg.addExpr(
+ new ValueVectorReadExpression(new TypedFieldId(expr.getMajorType(), count++)), false);
cg.setMappingSet(mainMapping);
- FunctionCall f = new FunctionCall(ComparatorFunctions.COMPARE_TO, ImmutableList.of((LogicalExpression) new HoldingContainerExpression(left), new HoldingContainerExpression(right)), ExpressionPosition.UNKNOWN);
+ FunctionCall f = new FunctionCall(ComparatorFunctions.COMPARE_TO, ImmutableList.of(
+ (LogicalExpression) new HoldingContainerExpression(left), new HoldingContainerExpression(right)),
+ ExpressionPosition.UNKNOWN);
CodeGenerator.HoldingContainer out = cg.addExpr(f, false);
JConditional jc = cg.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
- if(od.getDirection() == Order.Direction.ASC){
+ if (od.getDirection() == Order.Direction.ASC) {
jc._then()._return(out.getValue());
- }else{
+ } else {
jc._then()._return(out.getValue().minus());
}
}
cg.getEvalBlock()._return(JExpr.lit(0));
- SchemaPath outputPath = new SchemaPath(popConfig.getRef().getPath(), ExpressionPosition.UNKNOWN);
- MaterializedField outputField = MaterializedField.create(outputPath, Types.required(TypeProtos.MinorType.INT));
- ValueVector v = TypeHelper.getNewVector(outputField, context.getAllocator());
- container.add(v);
- this.allocationVectors.add(v);
+ container.add(this.partitionKeyVector);
container.buildSchema(batch.getSchema().getSelectionVectorMode());
-
+
try {
this.projector = context.getImplementationClass(cg);
- projector.setup(context, batch, this, transfers, partitionVectors, partitions, outputPath);
+ projector.setup(context, batch, this, transfers, partitionVectors, partitions, new SchemaPath(popConfig.getRef().getPath(), ExpressionPosition.UNKNOWN));
} 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/OutgoingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
index 081b4c297..60b395fda 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -17,6 +17,8 @@
*/
package org.apache.drill.exec.physical.impl.partitionsender;
+import io.netty.buffer.ByteBuf;
+
import java.util.Iterator;
import org.apache.drill.common.expression.SchemaPath;
@@ -24,8 +26,10 @@ import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.expr.TypeHelper;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.impl.SendingAccountor;
import org.apache.drill.exec.proto.ExecProtos;
import org.apache.drill.exec.proto.GeneralRPCProtos;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.record.BatchSchema;
import org.apache.drill.exec.record.FragmentWritableBatch;
import org.apache.drill.exec.record.RecordBatch;
@@ -35,14 +39,12 @@ import org.apache.drill.exec.record.VectorAccessible;
import org.apache.drill.exec.record.VectorContainer;
import org.apache.drill.exec.record.VectorWrapper;
import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.record.selection.SelectionVector4;
import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.rpc.data.DataTunnel;
import org.apache.drill.exec.vector.ValueVector;
import org.apache.drill.exec.vector.allocator.VectorAllocator;
-import org.apache.drill.exec.work.foreman.ErrorHelper;
+import org.apache.drill.exec.work.ErrorHelper;
import com.google.common.base.Preconditions;
@@ -54,25 +56,28 @@ import com.google.common.base.Preconditions;
public class OutgoingRecordBatch implements VectorAccessible {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OutgoingRecordBatch.class);
- private BitTunnel tunnel;
- private HashPartitionSender operator;
- private volatile boolean ok = true;
+ private final DataTunnel tunnel;
+ private final HashPartitionSender operator;
+ private final RecordBatch incoming;
+ private final FragmentContext context;
+ private final VectorContainer vectorContainer = new VectorContainer();
+ private final SendingAccountor sendCount;
+ private final int oppositeMinorFragmentId;
+
private boolean isLast = false;
- private RecordBatch incoming;
- private FragmentContext context;
+ private volatile boolean ok = true;
private BatchSchema outSchema;
- private VectorContainer vectorContainer;
private int recordCount;
private int recordCapacity;
- private int oppositeMinorFragmentId;
private static int DEFAULT_ALLOC_SIZE = 20000;
private static int DEFAULT_VARIABLE_WIDTH_SIZE = 2048;
- public OutgoingRecordBatch(HashPartitionSender operator, BitTunnel tunnel, RecordBatch incoming, FragmentContext context, int oppositeMinorFragmentId) {
+ public OutgoingRecordBatch(SendingAccountor sendCount, HashPartitionSender operator, DataTunnel tunnel, RecordBatch incoming, FragmentContext context, int oppositeMinorFragmentId) {
this.incoming = incoming;
this.context = context;
this.operator = operator;
this.tunnel = tunnel;
+ this.sendCount = sendCount;
this.oppositeMinorFragmentId = oppositeMinorFragmentId;
}
@@ -116,7 +121,8 @@ public class OutgoingRecordBatch implements VectorAccessible {
oppositeMinorFragmentId,
getWritableBatch());
- tunnel.sendRecordBatch(statusHandler, context, writableBatch);
+ tunnel.sendRecordBatch(statusHandler, writableBatch);
+ this.sendCount.increment();
} else {
logger.debug("Flush requested on an empty outgoing record batch" + (isLast ? " (last batch)" : ""));
if (isLast) {
@@ -128,7 +134,8 @@ public class OutgoingRecordBatch implements VectorAccessible {
operator.getOppositeMajorFragmentId(),
oppositeMinorFragmentId,
getWritableBatch());
- tunnel.sendRecordBatch(statusHandler, context, writableBatch);
+ tunnel.sendRecordBatch(statusHandler, writableBatch);
+ this.sendCount.increment();
vectorContainer.clear();
return true;
}
@@ -138,6 +145,7 @@ public class OutgoingRecordBatch implements VectorAccessible {
// NOTE: the value vector is directly referenced by generated code; therefore references
// must remain valid.
recordCount = 0;
+ vectorContainer.zeroVectors();
for (VectorWrapper<?> v : vectorContainer) {
logger.debug("Reallocating vv to capacity " + DEFAULT_ALLOC_SIZE + " after flush.");
VectorAllocator.getAllocator(v.getValueVector(), DEFAULT_VARIABLE_WIDTH_SIZE).alloc(DEFAULT_ALLOC_SIZE);
@@ -152,8 +160,8 @@ public class OutgoingRecordBatch implements VectorAccessible {
*/
public void initializeBatch() {
isLast = false;
+ vectorContainer.clear();
recordCapacity = DEFAULT_ALLOC_SIZE;
- vectorContainer = new VectorContainer();
SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(BatchSchema.SelectionVectorMode.NONE);
for (VectorWrapper<?> v : incoming) {
@@ -179,8 +187,9 @@ public class OutgoingRecordBatch implements VectorAccessible {
isLast = false;
recordCount = 0;
recordCapacity = 0;
- for (VectorWrapper<?> v : vectorContainer)
+ for (VectorWrapper<?> v : vectorContainer){
v.getValueVector().clear();
+ }
}
public void setIsLast() {
@@ -221,9 +230,16 @@ public class OutgoingRecordBatch implements VectorAccessible {
private StatusHandler statusHandler = new StatusHandler();
private class StatusHandler extends BaseRpcOutcomeListener<GeneralRPCProtos.Ack> {
RpcException ex;
+
+ @Override
+ public void success(Ack value, ByteBuf buffer) {
+ sendCount.decrement();
+ super.success(value, buffer);
+ }
@Override
public void failed(RpcException ex) {
+ sendCount.decrement();
logger.error("Failure while sending data to user.", ex);
ErrorHelper.logAndConvertError(context.getIdentity(), "Failure while sending fragment to client.", ex, logger);
ok = false;
@@ -232,4 +248,7 @@ public class OutgoingRecordBatch implements VectorAccessible {
}
+ public void clear(){
+ vectorContainer.clear();
+ }
}
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 409a9c37f..d88e5b58e 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
@@ -18,6 +18,7 @@
package org.apache.drill.exec.physical.impl.partitionsender;
import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.drill.common.expression.ErrorCollector;
import org.apache.drill.common.expression.ErrorCollectorImpl;
@@ -30,8 +31,10 @@ import org.apache.drill.exec.expr.TypeHelper;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.config.HashPartitionSender;
import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.physical.impl.SendingAccountor;
import org.apache.drill.exec.physical.impl.filter.ReturnValueExpression;
import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.record.TypedFieldId;
import org.apache.drill.exec.record.VectorWrapper;
@@ -54,6 +57,9 @@ public class PartitionSenderRootExec implements RootExec {
private Partitioner partitioner;
private FragmentContext context;
private boolean ok = true;
+ private AtomicLong batchesSent = new AtomicLong(0);
+ private final SendingAccountor sendCount = new SendingAccountor();
+
public PartitionSenderRootExec(FragmentContext context,
RecordBatch incoming,
@@ -65,8 +71,9 @@ public class PartitionSenderRootExec implements RootExec {
this.outgoing = new OutgoingRecordBatch[operator.getDestinations().size()];
int fieldId = 0;
for (CoordinationProtos.DrillbitEndpoint endpoint : operator.getDestinations()) {
- outgoing[fieldId] = new OutgoingRecordBatch(operator,
- context.getCommunicator().getTunnel(endpoint),
+ FragmentHandle opposite = context.getHandle().toBuilder().setMajorFragmentId(operator.getOppositeMajorFragmentId()).setMinorFragmentId(fieldId).build();
+ outgoing[fieldId] = new OutgoingRecordBatch(sendCount, operator,
+ context.getDataTunnel(endpoint, opposite),
incoming,
context,
fieldId);
@@ -79,6 +86,7 @@ public class PartitionSenderRootExec implements RootExec {
if (!ok) {
stop();
+
return false;
}
@@ -119,8 +127,8 @@ public class PartitionSenderRootExec implements RootExec {
}
case OK:
partitioner.partitionBatch(incoming);
- context.batchesCompleted.inc(1);
- context.recordsCompleted.inc(incoming.getRecordCount());
+ context.getStats().batchesCompleted.inc(1);
+ context.getStats().recordsCompleted.inc(incoming.getRecordCount());
return true;
case NOT_YET:
default:
@@ -128,10 +136,7 @@ public class PartitionSenderRootExec implements RootExec {
}
}
- public void stop() {
- ok = false;
- incoming.kill();
- }
+
private void generatePartitionFunction() throws SchemaChangeException {
@@ -285,4 +290,14 @@ public class PartitionSenderRootExec implements RootExec {
}
}
}
+
+ public void stop() {
+ logger.debug("Partition sender stopping.");
+ ok = false;
+ for(OutgoingRecordBatch b : outgoing){
+ b.clear();
+ }
+ incoming.cleanup();
+ sendCount.waitForSendComplete();
+ }
}
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 e01fb7404..f8625109f 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
@@ -36,9 +36,12 @@ import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
import org.apache.drill.exec.expr.HoldingContainerExpression;
import org.apache.drill.exec.expr.fn.impl.ComparatorFunctions;
import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.physical.config.Sort;
-import org.apache.drill.exec.record.*;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.WritableBatch;
import org.apache.drill.exec.record.selection.SelectionVector2;
import org.apache.drill.exec.record.selection.SelectionVector4;
@@ -56,24 +59,25 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
private static long MAX_SORT_BYTES = 8l * 1024 * 1024 * 1024;
private final RecordBatch incoming;
- private SortRecordBatchBuilder builder;
- private SelectionVector4 sv4;
+ private final SortRecordBatchBuilder builder;
private Sorter sorter;
private BatchSchema schema;
public SortBatch(Sort popConfig, FragmentContext context, RecordBatch incoming) {
super(popConfig, context);
this.incoming = incoming;
+ this.builder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES);
}
@Override
public int getRecordCount() {
- return sv4.getCount();
+ return builder.getSv4().getCount();
}
@Override
public void kill() {
incoming.kill();
+ cleanup();
}
@Override
@@ -83,22 +87,22 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
@Override
public SelectionVector4 getSelectionVector4() {
- return sv4;
+ return builder.getSv4();
}
@Override
- protected void cleanup() {
+ public void cleanup() {
super.cleanup();
- container.zeroVectors();;
- sv4.clear();
+ incoming.cleanup();
+ builder.clear();
}
@Override
public IterOutcome next() {
- if(builder != null){
- if(sv4.next()){
+ if(schema != null){
+ if(getSelectionVector4().next()){
return IterOutcome.OK;
}else{
return IterOutcome.NONE;
@@ -113,14 +117,14 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
case NONE:
break outer;
case NOT_YET:
+ throw new UnsupportedOperationException();
case STOP:
- container.zeroVectors();
+ cleanup();
return upstream;
case OK_NEW_SCHEMA:
// only change in the case that the schema truly changes. Artificial schema changes are ignored.
if(!incoming.getSchema().equals(schema)){
- if (builder != null) throw new UnsupportedOperationException("Sort doesn't currently support sorts with changing schemas.");
- builder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES, container);
+ if (schema != null) throw new UnsupportedOperationException("Sort doesn't currently support sorts with changing schemas.");
this.schema = incoming.getSchema();
}
// fall through.
@@ -134,19 +138,16 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
}
}
- if (builder == null){
+ if (schema == null){
// builder may be null at this point if the first incoming batch is empty
return IterOutcome.NONE;
}
- builder.build(context);
- sv4 = builder.getSv4();
+ builder.build(context, container);
sorter = createNewSorter();
- sorter.setup(context, this.getSelectionVector4(), this.container);
- long t1 = System.nanoTime();
- sorter.sort(sv4, container);
- logger.debug("Sorted {} records in {} micros.", sv4.getTotalCount(), (System.nanoTime() - t1)/1000);
+ sorter.setup(context, getSelectionVector4(), this.container);
+ sorter.sort(getSelectionVector4(), this.container);
return IterOutcome.OK_NEW_SCHEMA;
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 cb55a01f3..d8bbc6a54 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
@@ -17,9 +17,6 @@
*/
package org.apache.drill.exec.physical.impl.sort;
-import com.google.common.collect.Lists;
-import io.netty.buffer.ByteBuf;
-
import java.util.ArrayList;
import java.util.List;
@@ -27,18 +24,22 @@ import org.apache.drill.exec.exception.SchemaChangeException;
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.*;
+import org.apache.drill.exec.record.BatchSchema;
import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
import org.apache.drill.exec.record.selection.SelectionVector4;
import org.apache.drill.exec.vector.ValueVector;
import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Lists;
public class SortRecordBatchBuilder {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SortRecordBatchBuilder.class);
private final ArrayListMultimap<BatchSchema, RecordBatchData> batches = ArrayListMultimap.create();
- private final VectorContainer container;
private int recordCount;
private long runningBytes;
@@ -47,10 +48,9 @@ public class SortRecordBatchBuilder {
private SelectionVector4 sv4;
final PreAllocator svAllocator;
- public SortRecordBatchBuilder(BufferAllocator a, long maxBytes, VectorContainer container){
+ public SortRecordBatchBuilder(BufferAllocator a, long maxBytes){
this.maxBytes = maxBytes;
- this.svAllocator = a.getPreAllocator();
- this.container = container;
+ this.svAllocator = a.getNewPreAllocator();
}
private long getSize(VectorAccessible batch){
@@ -89,8 +89,8 @@ public class SortRecordBatchBuilder {
return true;
}
- public void build(FragmentContext context) throws SchemaChangeException{
- container.clear();
+ public void build(FragmentContext context, 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);
assert batches.keySet().size() > 0;
@@ -138,18 +138,25 @@ public class SortRecordBatchBuilder {
for(MaterializedField f : vectors.keySet()){
List<ValueVector> v = vectors.get(f);
- container.addHyperList(v);
+ outputContainer.addHyperList(v, false);
}
- container.buildSchema(SelectionVectorMode.FOUR_BYTE);
+ outputContainer.buildSchema(SelectionVectorMode.FOUR_BYTE);
}
public SelectionVector4 getSv4() {
return sv4;
}
- public List<VectorContainer> getContainers() {
- ArrayList containerList = Lists.newArrayList();
+ public void clear(){
+ for(RecordBatchData d : batches.values()){
+ d.container.clear();
+ }
+ if(sv4 != null) sv4.clear();
+ }
+
+ public List<VectorContainer> getHeldRecordBatches() {
+ ArrayList<VectorContainer> containerList = Lists.newArrayList();
for (BatchSchema bs : batches.keySet()) {
for (RecordBatchData bd : batches.get(bs)) {
VectorContainer c = bd.getContainer();
@@ -157,6 +164,7 @@ public class SortRecordBatchBuilder {
containerList.add(c);
}
}
+ batches.clear();
return containerList;
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java
index a8b1eecee..6d1273139 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java
@@ -22,6 +22,7 @@ import javax.inject.Named;
import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorWrapper;
import org.apache.drill.exec.record.selection.SelectionVector2;
import org.apache.drill.exec.vector.allocator.VectorAllocator;
@@ -30,11 +31,13 @@ public abstract class CopierTemplate2 implements Copier{
private SelectionVector2 sv2;
private VectorAllocator[] allocators;
+ private RecordBatch incoming;
@Override
public void setupRemover(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, VectorAllocator[] allocators) throws SchemaChangeException{
this.allocators = allocators;
this.sv2 = incoming.getSelectionVector2();
+ this.incoming = incoming;
doSetup(context, incoming, outgoing);
}
@@ -53,6 +56,10 @@ public abstract class CopierTemplate2 implements Copier{
for(int svIndex = 0; svIndex < recordCount; svIndex++, outgoingPosition++){
doEval(sv2.getIndex(svIndex), outgoingPosition);
}
+// logger.debug("This: {}, Incoming: {}", System.identityHashCode(this), incoming);
+ for(VectorWrapper<?> v : incoming){
+ v.clear();
+ }
}
public abstract void doSetup(@Named("context") FragmentContext context, @Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate4.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate4.java
index ea352723e..2a3a2e28b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate4.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate4.java
@@ -22,6 +22,7 @@ import javax.inject.Named;
import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorWrapper;
import org.apache.drill.exec.record.selection.SelectionVector2;
import org.apache.drill.exec.record.selection.SelectionVector4;
import org.apache.drill.exec.vector.allocator.VectorAllocator;
@@ -31,7 +32,7 @@ public abstract class CopierTemplate4 implements Copier{
private SelectionVector4 sv4;
private VectorAllocator[] allocators;
-
+ private RecordBatch incoming;
private void allocateVectors(int recordCount){
for(VectorAllocator a : allocators){
@@ -42,6 +43,7 @@ public abstract class CopierTemplate4 implements Copier{
@Override
public void setupRemover(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, VectorAllocator[] allocators) throws SchemaChangeException{
this.allocators = allocators;
+ this.incoming = incoming;
this.sv4 = incoming.getSelectionVector4();
doSetup(context, incoming, outgoing);
}
@@ -49,6 +51,7 @@ public abstract class CopierTemplate4 implements Copier{
@Override
public void copyRecords(){
+// logger.debug("Copying records.");
final int recordCount = sv4.getCount();
allocateVectors(recordCount);
int outgoingPosition = 0;
@@ -56,6 +59,11 @@ public abstract class CopierTemplate4 implements Copier{
int deRefIndex = sv4.get(svIndex);
doEval(deRefIndex, outgoingPosition);
}
+
+ for(VectorWrapper<?> v : incoming){
+ v.clear();
+ }
+
}
public abstract void doSetup(@Named("context") FragmentContext context, @Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing);
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 663f4c3d3..123f022a2 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
@@ -54,6 +54,7 @@ public class RemovingRecordBatch extends AbstractSingleRecordBatch<SelectionVect
public RemovingRecordBatch(SelectionVectorRemover popConfig, FragmentContext context, RecordBatch incoming) {
super(popConfig, context, incoming);
+ logger.debug("Created.");
}
@Override
@@ -64,7 +65,7 @@ public class RemovingRecordBatch extends AbstractSingleRecordBatch<SelectionVect
@Override
protected void setupNewSchema() throws SchemaChangeException {
container.clear();
-
+
switch(incoming.getSchema().getSelectionVectorMode()){
case NONE:
this.copier = getStraightCopier();
@@ -93,6 +94,9 @@ public class RemovingRecordBatch extends AbstractSingleRecordBatch<SelectionVect
}
}
+ public void cleanup(){
+ super.cleanup();
+ }
private class StraightCopier implements Copier{
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 c138ababb..c9a73f90a 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
@@ -1,5 +1,5 @@
/**
- * Licensed to the Apache Software Foundation (ASF) under one
+ * Licensed to th7e 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
@@ -18,38 +18,25 @@
package org.apache.drill.exec.physical.impl.trace;
-import java.io.File;
-import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
-import java.nio.channels.FileChannel;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Formatter;
-import com.google.common.collect.Iterators;
-import io.netty.buffer.CompositeByteBuf;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.cache.VectorAccessibleSerializable;
import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.config.Trace;
-import org.apache.drill.exec.record.*;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.vector.ValueVector;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.helper.QueryIdHelper;
-import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
-
-import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.record.AbstractSingleRecordBatch;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -174,7 +161,7 @@ public class TraceRecordBatch extends AbstractSingleRecordBatch<Trace> {
}
@Override
- protected void cleanup() {
+ public void cleanup() {
/* Release the selection vector */
if (sv != null)
sv.clear();
@@ -185,6 +172,7 @@ public class TraceRecordBatch extends AbstractSingleRecordBatch<Trace> {
} catch (IOException e) {
logger.error("Unable to close file descriptors for file: " + getFileName());
}
+ super.cleanup();
}
}
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 d6f08f191..8552465b1 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
@@ -113,4 +113,9 @@ public class IteratorValidatorBatchIterator implements RecordBatch{
validateReadState();
return incoming.getWritableBatch();
}
+
+ @Override
+ public void cleanup() {
+ incoming.cleanup();
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
index 8262691bb..ffb4c5b30 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
@@ -27,9 +27,9 @@ import org.apache.drill.exec.physical.base.FragmentRoot;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.planner.fragment.Materializer.IndexedFragmentNode;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.work.QueryWorkUnit;
@@ -128,11 +128,11 @@ public class SimpleParallelizer {
.build();
if (isRootNode) {
- logger.debug("Root fragment {}", fragment);
+ logger.debug("Root fragment:\n {}", fragment);
rootFragment = fragment;
rootOperator = root;
} else {
- logger.debug("Remote fragment {}", fragment);
+ logger.debug("Remote fragment:\n {}", fragment);
fragments.add(fragment);
}
}
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/helper/QueryIdHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/proto/helper/QueryIdHelper.java
index 749b3d5d0..749b3d5d0 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/helper/QueryIdHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/proto/helper/QueryIdHelper.java
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 ec2d246a4..d4f458f33 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
@@ -59,16 +59,16 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
@Override
public void kill() {
- container.zeroVectors();
killIncoming();
cleanup();
}
protected abstract void killIncoming();
- protected void cleanup(){
+ public void cleanup(){
+ container.clear();
}
-
+
@Override
public SelectionVector2 getSelectionVector2() {
throw new UnsupportedOperationException();
@@ -92,7 +92,10 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
@Override
public WritableBatch getWritableBatch() {
- return WritableBatch.get(this);
+// logger.debug("Getting writable batch.");
+ WritableBatch batch = WritableBatch.get(this);
+ return batch;
+
}
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 1639940da..7b832e43a 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
@@ -22,7 +22,7 @@ import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.base.PhysicalOperator;
public abstract class AbstractSingleRecordBatch<T extends PhysicalOperator> extends AbstractRecordBatch<T> {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSingleRecordBatch.class);
+ final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(this.getClass());
protected final RecordBatch incoming;
private boolean first = true;
@@ -46,7 +46,6 @@ public abstract class AbstractSingleRecordBatch<T extends PhysicalOperator> exte
case NONE:
case NOT_YET:
case STOP:
- container.zeroVectors();
cleanup();
return upstream;
case OK_NEW_SCHEMA:
@@ -67,6 +66,13 @@ public abstract class AbstractSingleRecordBatch<T extends PhysicalOperator> exte
}
}
+ @Override
+ public void cleanup() {
+// logger.debug("Cleaning up.");
+ incoming.cleanup();
+ super.cleanup();
+ }
+
protected abstract void setupNewSchema() throws SchemaChangeException;
protected abstract void doWork();
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
index 41ad8b74f..33fd5a268 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
@@ -19,8 +19,8 @@ package org.apache.drill.exec.record;
import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
index d6d49c9f6..44b935b8d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
@@ -88,7 +88,7 @@ public class HyperVectorWrapper<T extends ValueVector> implements VectorWrapper<
// return new HyperVectorWrapper<T>(f, newVectors);
}
- public static <T extends ValueVector> HyperVectorWrapper<T> create(MaterializedField f, T[] v){
- return new HyperVectorWrapper<T>(f, v);
+ public static <T extends ValueVector> HyperVectorWrapper<T> create(MaterializedField f, T[] v, boolean releasable){
+ return new HyperVectorWrapper<T>(f, v, releasable);
}
}
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 74974750d..a7d3f4b59 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
@@ -19,18 +19,21 @@ package org.apache.drill.exec.record;
import io.netty.buffer.ByteBuf;
-import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
+import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
+import org.apache.drill.exec.rpc.RemoteConnection;
public class RawFragmentBatch {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RawFragmentBatch.class);
+ final RemoteConnection connection;
final FragmentRecordBatch header;
final ByteBuf body;
- public RawFragmentBatch(FragmentRecordBatch header, ByteBuf body) {
+ public RawFragmentBatch(RemoteConnection connection, FragmentRecordBatch header, ByteBuf body) {
super();
this.header = header;
this.body = body;
+ this.connection = connection;
if(body != null) body.retain();
}
@@ -51,4 +54,9 @@ public class RawFragmentBatch {
if(body != null) body.release();
}
+ public RemoteConnection getConnection() {
+ return connection;
+ }
+
+
}
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 c13838cc6..b41b733fb 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
@@ -110,5 +110,7 @@ public interface RecordBatch extends VectorAccessible {
* @return
*/
public WritableBatch getWritableBatch();
+
+ public void cleanup();
}
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 f19184f2b..bbeae4f42 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
@@ -153,6 +153,8 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
return schema;
}
-
+ public void clear(){
+ container.clear();
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
index 462c00a34..8d952e32f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
@@ -32,7 +32,6 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorContainer.class);
private final List<VectorWrapper<?>> wrappers = Lists.newArrayList();
- private final List<VectorWrapper<?>> oldWrappers = Lists.newArrayList();
private BatchSchema schema;
private int recordCount = -1;
@@ -50,12 +49,16 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
// }
public void addHyperList(List<ValueVector> vectors) {
+ addHyperList(vectors, true);
+ }
+
+ public void addHyperList(List<ValueVector> vectors, boolean releasable) {
schema = null;
ValueVector[] vv = new ValueVector[vectors.size()];
for (int i = 0; i < vv.length; i++) {
vv[i] = vectors.get(i);
}
- add(vv);
+ add(vv, releasable);
}
/**
@@ -95,6 +98,9 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
}
public void add(ValueVector[] hyperVector) {
+
+ }
+ public void add(ValueVector[] hyperVector, boolean releasable) {
assert hyperVector.length != 0;
schema = null;
Class<?> clazz = hyperVector[0].getClass();
@@ -103,7 +109,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
c[i] = hyperVector[i];
}
// todo: work with a merged schema.
- wrappers.add(HyperVectorWrapper.create(hyperVector[0].getField(), c));
+ wrappers.add(HyperVectorWrapper.create(hyperVector[0].getField(), c, releasable));
}
public void remove(ValueVector v) {
@@ -166,10 +172,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
}
public void clear() {
- // TODO: figure out a better approach for this.
- // we don't clear schema because we want empty batches to carry previous schema to avoid extra schema update for no
- // data.
- // schema = null;
+ schema = null;
zeroVectors();
wrappers.clear();
}
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 945df4187..288aa7f77 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
@@ -41,7 +41,6 @@ public class WritableBatch {
private boolean cleared = false;
private WritableBatch(RecordBatchDef def, List<ByteBuf> buffers) {
- logger.debug("Created new writable batch with def {} and buffers {}", def, buffers);
this.def = def;
this.buffers = buffers.toArray(new ByteBuf[buffers.size()]);
}
@@ -106,6 +105,7 @@ public class WritableBatch {
}
public void clear() {
+ if(cleared) return;
for (ByteBuf buf : buffers) {
buf.release();
}
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 6ff6cf270..f7b5155b6 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
@@ -106,7 +106,7 @@ public class SelectionVector2 implements Closeable{
}
public void setRecordCount(int recordCount){
- logger.debug("Seting record count to {}", recordCount);
+// logger.debug("Seting record count to {}", recordCount);
this.recordCount = recordCount;
}
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 4533cc2f3..ac9dcda8b 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
@@ -20,11 +20,12 @@ package org.apache.drill.exec.record.selection;
import io.netty.buffer.ByteBuf;
import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.record.DeadBuf;
public class SelectionVector4 {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector4.class);
- private final ByteBuf vector;
+ private ByteBuf data;
private int recordCount;
private int start;
private int length;
@@ -34,7 +35,7 @@ public class SelectionVector4 {
this.recordCount = recordCount;
this.start = 0;
this.length = Math.min(batchRecordCount, recordCount);
- this.vector = vector;
+ this.data = vector;
}
public int getTotalCount(){
@@ -51,14 +52,14 @@ public class SelectionVector4 {
}
public void set(int index, int compound){
- vector.setInt(index*4, compound);
+ data.setInt(index*4, compound);
}
public void set(int index, int recordBatch, int recordIndex){
- vector.setInt(index*4, (recordBatch << 16) | (recordIndex & 65535));
+ data.setInt(index*4, (recordBatch << 16) | (recordIndex & 65535));
}
public int get(int index){
- return vector.getInt( (start+index)*4);
+ return data.getInt( (start+index)*4);
}
/**
@@ -68,8 +69,8 @@ public class SelectionVector4 {
*/
public SelectionVector4 createNewWrapperCurrent(){
try {
- vector.retain();
- SelectionVector4 sv4 = new SelectionVector4(vector, length, length);
+ data.retain();
+ SelectionVector4 sv4 = new SelectionVector4(data, length, length);
sv4.start = this.start;
return sv4;
} catch (SchemaChangeException e) {
@@ -98,7 +99,10 @@ public class SelectionVector4 {
public void clear(){
start = 0;
length = 0;
- this.vector.clear();
+ if (data != DeadBuf.DEAD_BUFFER) {
+ data.release();
+ data = DeadBuf.DEAD_BUFFER;
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
index 066af0608..4a43a9948 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
@@ -30,6 +30,7 @@ import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.util.concurrent.GenericFutureListener;
+import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.rpc.RpcConnectionHandler.FailureType;
import com.google.protobuf.Internal.EnumLite;
@@ -60,16 +61,17 @@ public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection
.option(ChannelOption.ALLOCATOR, alloc) //
.option(ChannelOption.SO_RCVBUF, 1 << 17) //
.option(ChannelOption.SO_SNDBUF, 1 << 17) //
+ .option(ChannelOption.TCP_NODELAY, true)
.handler(new ChannelInitializer<SocketChannel>() {
@Override
protected void initChannel(SocketChannel ch) throws Exception {
- logger.debug("initializing client connection.");
+// logger.debug("initializing client connection.");
connection = initRemoteConnection(ch);
ch.closeFuture().addListener(getCloseHandler(connection));
ch.pipeline().addLast( //
- new ZeroCopyProtobufLengthDecoder(), //
+ getDecoder(connection.getAllocator()), //
new RpcDecoder("c-" + rpcConfig.getName()), //
new RpcEncoder("c-" + rpcConfig.getName()), //
new ClientHandshakeHandler(), //
@@ -82,6 +84,8 @@ public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection
;
}
+
+ public abstract ProtobufLengthDecoder getDecoder(BufferAllocator allocator);
public boolean isActive(){
return connection.getChannel().isActive() ;
@@ -94,18 +98,12 @@ public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection
return new ChannelClosedHandler();
}
- protected final <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFutureImpl<RECEIVE> send(
- T connection, T rpcType, SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
- throw new UnsupportedOperationException(
- "This shouldn't be used in client mode as a client only has a single connection.");
- }
-
- protected <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> listener,
- T rpcType, SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
+ public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> listener,
+ T rpcType, SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
super.send(listener, connection, rpcType, protobufBody, clazz, dataBodies);
}
- protected <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(T rpcType,
+ public <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(T rpcType,
SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
return super.send(connection, rpcType, protobufBody, clazz, dataBodies);
}
@@ -146,7 +144,8 @@ public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection
try {
future.get();
if (future.isSuccess()) {
- send(handshakeSendHandler, handshakeType, handshakeValue, responseClass);
+ // send a handshake on the current thread. This is the only time we will send from within the event thread. We can do this because the connection will not be backed up.
+ send(handshakeSendHandler, connection, handshakeType, handshakeValue, responseClass, true);
} else {
l.connectionFailed(FailureType.CONNECTION, new RpcException("General connection failure."));
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
index df8638b24..a0b5cfd65 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
@@ -18,24 +18,27 @@
package org.apache.drill.exec.rpc;
import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.EventLoopGroup;
import io.netty.util.concurrent.GenericFutureListener;
+import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.rpc.BasicClientWithConnection.ServerConnection;
+import com.google.protobuf.Internal.EnumLite;
import com.google.protobuf.MessageLite;
import com.google.protobuf.Parser;
-import com.google.protobuf.Internal.EnumLite;
public abstract class BasicClientWithConnection<T extends EnumLite, HANDSHAKE_SEND extends MessageLite, HANDSHAKE_RESPONSE extends MessageLite> extends BasicClient<T, ServerConnection, HANDSHAKE_SEND, HANDSHAKE_RESPONSE>{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicClientWithConnection.class);
- public BasicClientWithConnection(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, T handshakeType,
+ private BufferAllocator alloc;
+
+ public BasicClientWithConnection(RpcConfig rpcMapping, BufferAllocator alloc, EventLoopGroup eventLoopGroup, T handshakeType,
Class<HANDSHAKE_RESPONSE> responseClass, Parser<HANDSHAKE_RESPONSE> handshakeParser) {
- super(rpcMapping, alloc, eventLoopGroup, handshakeType, responseClass, handshakeParser);
+ super(rpcMapping, alloc.getUnderlyingAllocator(), eventLoopGroup, handshakeType, responseClass, handshakeParser);
+ this.alloc = alloc;
}
@Override
@@ -52,15 +55,25 @@ public abstract class BasicClientWithConnection<T extends EnumLite, HANDSHAKE_SE
@Override
public ServerConnection initRemoteConnection(Channel channel) {
- return new ServerConnection(channel);
+ return new ServerConnection(channel, alloc);
}
public static class ServerConnection extends RemoteConnection{
- public ServerConnection(Channel channel) {
+ private final BufferAllocator alloc;
+
+ public ServerConnection(Channel channel, BufferAllocator alloc) {
super(channel);
+ this.alloc = alloc;
}
+ @Override
+ public BufferAllocator getAllocator() {
+ return alloc;
+ }
+
+
+
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
index a947f0af5..f5e77f10c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
@@ -18,7 +18,9 @@
package org.apache.drill.exec.rpc;
import io.netty.bootstrap.ServerBootstrap;
+import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
@@ -32,6 +34,7 @@ import java.io.IOException;
import java.net.BindException;
import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
import com.google.protobuf.Internal.EnumLite;
@@ -42,7 +45,7 @@ import com.google.protobuf.Parser;
* A server is bound to a port and is responsible for responding to various type of requests. In some cases, the inbound
* requests will generate more than one outbound request.
*/
-public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection> extends RpcBus<T, C>{
+public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection> extends RpcBus<T, C> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicServer.class);
private ServerBootstrap b;
@@ -52,10 +55,11 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
public BasicServer(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
super(rpcMapping);
this.eventLoopGroup = eventLoopGroup;
-
+
b = new ServerBootstrap() //
.channel(NioServerSocketChannel.class) //
.option(ChannelOption.SO_BACKLOG, 100) //
+ .option(ChannelOption.TCP_NODELAY, true)
.option(ChannelOption.SO_RCVBUF, 1 << 17) //
.option(ChannelOption.SO_SNDBUF, 1 << 17) //
.group(eventLoopGroup) //
@@ -64,30 +68,30 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
.childHandler(new ChannelInitializer<SocketChannel>() {
@Override
protected void initChannel(SocketChannel ch) throws Exception {
-
+// logger.debug("Starting initialization of server connection.");
C connection = initRemoteConnection(ch);
ch.closeFuture().addListener(getCloseHandler(connection));
-
+
ch.pipeline().addLast( //
- new ZeroCopyProtobufLengthDecoder(), //
+ getDecoder(connection.getAllocator()), //
new RpcDecoder("s-" + rpcConfig.getName()), //
new RpcEncoder("s-" + rpcConfig.getName()), //
- getHandshakeHandler(connection),
- new InboundHandler(connection), //
+ getHandshakeHandler(connection), new InboundHandler(connection), //
new RpcExceptionHandler() //
- );
+ );
connect = true;
-
+// logger.debug("Server connection initialization completed.");
}
});
}
-
+
+ public abstract ProtobufLengthDecoder getDecoder(BufferAllocator allocator);
+
@Override
public boolean isClient() {
return false;
}
-
protected abstract ServerHandshakeHandler<?> getHandshakeHandler(C connection);
protected static abstract class ServerHandshakeHandler<T extends MessageLite> extends AbstractHandshakeHandler<T> {
@@ -98,17 +102,44 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
@Override
protected final void consumeHandshake(ChannelHandlerContext ctx, T inbound) throws Exception {
- OutboundRpcMessage msg = new OutboundRpcMessage(RpcMode.RESPONSE, this.handshakeType, coordinationId, getHandshakeResponse(inbound));
+ OutboundRpcMessage msg = new OutboundRpcMessage(RpcMode.RESPONSE, this.handshakeType, coordinationId,
+ getHandshakeResponse(inbound));
ctx.writeAndFlush(msg);
}
-
+
public abstract MessageLite getHandshakeResponse(T inbound) throws Exception;
-
+
}
-
-
- public int bind(final int initialPort) throws InterruptedException, DrillbitStartupException{
- int port = initialPort-1;
+
+ @Override
+ protected MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+ return null;
+ }
+
+ @Override
+ protected Response handle(C connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ return null;
+ }
+
+ @Override
+ public <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(C connection, T rpcType,
+ SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
+ return super.send(connection, rpcType, protobufBody, clazz, dataBodies);
+ }
+
+ @Override
+ public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> listener,
+ C connection, T rpcType, SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
+ super.send(listener, connection, rpcType, protobufBody, clazz, dataBodies);
+ }
+
+ @Override
+ public C initRemoteConnection(Channel channel) {
+ return null;
+ }
+
+ public int bind(final int initialPort) throws InterruptedException, DrillbitStartupException {
+ int port = initialPort - 1;
while (true) {
try {
b.bind(++port).sync();
@@ -119,17 +150,15 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
throw new DrillbitStartupException("Could not bind Drillbit", e);
}
}
-
+
connect = !connect;
logger.debug("Server started on port {} of type {} ", port, this.getClass().getSimpleName());
- return port;
+ return port;
}
@Override
public void close() throws IOException {
eventLoopGroup.shutdownGracefully();
}
-
-
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
index 6e1cf7990..2bce738dd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
@@ -78,7 +78,7 @@ public class CoordinationQueue {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
- connection.releasePermit();
+
if(!future.isSuccess()){
removeFromMap(coordinationId);
if(future.channel().isActive()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/FutureBitCommand.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/FutureBitCommand.java
index f6c2660f0..fe5d12db9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/FutureBitCommand.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/FutureBitCommand.java
@@ -15,19 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc;
import io.netty.buffer.ByteBuf;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.RpcCheckedFuture;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.RpcOutcomeListener;
-
import com.google.common.util.concurrent.SettableFuture;
import com.google.protobuf.MessageLite;
-public abstract class FutureBitCommand<T extends MessageLite> implements BitCommand<T> {
+public abstract class FutureBitCommand<T extends MessageLite, C extends RemoteConnection> implements RpcCommand<T,C> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FutureBitCommand.class);
protected final SettableFuture<T> settableFuture;
@@ -38,16 +33,16 @@ public abstract class FutureBitCommand<T extends MessageLite> implements BitComm
this.parentFuture = new RpcCheckedFuture<T>(settableFuture);
}
- public abstract void doRpcCall(RpcOutcomeListener<T> outcomeListener, BitConnection connection);
+ public abstract void doRpcCall(RpcOutcomeListener<T> outcomeListener, C connection);
@Override
- public void connectionAvailable(BitConnection connection) {
+ public void connectionAvailable(C connection) {
doRpcCall(new DeferredRpcOutcome(), connection);
}
@Override
- public void connectionSucceeded(BitConnection connection) {
+ public void connectionSucceeded(C connection) {
connectionAvailable(connection);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListeningBitCommand.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ListeningCommand.java
index fa036b6bd..acb696c54 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListeningBitCommand.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ListeningCommand.java
@@ -15,34 +15,31 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc;
import io.netty.buffer.ByteBuf;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.RpcOutcomeListener;
-
import com.google.protobuf.MessageLite;
-public abstract class ListeningBitCommand<T extends MessageLite> implements BitCommand<T> {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ListeningBitCommand.class);
+public abstract class ListeningCommand<T extends MessageLite, C extends RemoteConnection> implements RpcCommand<T, C> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ListeningCommand.class);
private final RpcOutcomeListener<T> listener;
- public ListeningBitCommand(RpcOutcomeListener<T> listener) {
+ public ListeningCommand(RpcOutcomeListener<T> listener) {
this.listener = listener;
}
- public abstract void doRpcCall(RpcOutcomeListener<T> outcomeListener, BitConnection connection);
+ public abstract void doRpcCall(RpcOutcomeListener<T> outcomeListener, C connection);
@Override
- public void connectionAvailable(BitConnection connection) {
+ public void connectionAvailable(C connection) {
doRpcCall(new DeferredRpcOutcome(), connection);
}
@Override
- public void connectionSucceeded(BitConnection connection) {
+ public void connectionSucceeded(C connection) {
connectionAvailable(connection);
}
@@ -60,7 +57,6 @@ public abstract class ListeningBitCommand<T extends MessageLite> implements BitC
}
-
@Override
public void connectionFailed(FailureType type, Throwable t) {
listener.failed(RpcException.mapException(
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ProtobufLengthDecoder.java
index cff1a9047..6fef7e51e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ProtobufLengthDecoder.java
@@ -24,17 +24,27 @@ import io.netty.handler.codec.CorruptedFrameException;
import java.util.List;
+import org.apache.drill.exec.memory.BufferAllocator;
+
import com.google.protobuf.CodedInputStream;
/**
* Modified version of ProtobufVarint32FrameDecoder that avoids bytebuf copy.
*/
-public class ZeroCopyProtobufLengthDecoder extends ByteToMessageDecoder {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZeroCopyProtobufLengthDecoder.class);
+public class ProtobufLengthDecoder extends ByteToMessageDecoder {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtobufLengthDecoder.class);
+
+
+ private BufferAllocator allocator;
+
+ public ProtobufLengthDecoder(BufferAllocator allocator) {
+ super();
+ this.allocator = allocator;
+ }
+
@Override
protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
-
if (!ctx.channel().isOpen()) {
if (in.readableBytes() > 0)
logger.info("Channel is closed, discarding remaining {} byte(s) in buffer.", in.readableBytes());
@@ -68,13 +78,21 @@ public class ZeroCopyProtobufLengthDecoder extends ByteToMessageDecoder {
} else {
// need to make buffer copy, otherwise netty will try to refill this buffer if we move the readerIndex forward...
// TODO: Can we avoid this copy?
- ByteBuf outBuf = in.copy(in.readerIndex(), length);
+ ByteBuf outBuf = allocator.buffer(length);
+ if(outBuf == null){
+ logger.debug("Failure allocating buffer on incoming stream due to memory limits.");
+ in.resetReaderIndex();
+ return;
+ }
+ outBuf.writeBytes(in, in.readerIndex(), length);
+
in.skipBytes(length);
if (RpcConstants.EXTRA_DEBUGGING)
logger.debug(String.format(
"ReaderIndex is %d after length header of %d bytes and frame body of length %d bytes.",
in.readerIndex(), i + 1, length));
+
out.add(outBuf);
return;
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ReconnectingConnection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ReconnectingConnection.java
new file mode 100644
index 000000000..4cd41558f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ReconnectingConnection.java
@@ -0,0 +1,242 @@
+/**
+ * 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.rpc;
+
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import java.io.Closeable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.protobuf.MessageLite;
+
+/**
+ * Manager all connections between two particular bits.
+ */
+public abstract class ReconnectingConnection<CONNECTION_TYPE extends RemoteConnection, OUTBOUND_HANDSHAKE extends MessageLite>
+ implements Closeable {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ReconnectingConnection.class);
+
+ private final AtomicReference<CONNECTION_TYPE> connectionHolder = new AtomicReference<CONNECTION_TYPE>();
+ private final String host;
+ private final int port;
+ private final OUTBOUND_HANDSHAKE handshake;
+
+ public ReconnectingConnection(OUTBOUND_HANDSHAKE handshake, String host, int port) {
+ Preconditions.checkNotNull(host);
+ Preconditions.checkArgument(port > 0);
+ this.host = host;
+ this.port = port;
+ this.handshake = handshake;
+ }
+
+ protected abstract BasicClient<?, CONNECTION_TYPE, OUTBOUND_HANDSHAKE, ?> getNewClient();
+
+ public <R extends MessageLite, C extends RpcCommand<R, CONNECTION_TYPE>> void runCommand(C cmd) {
+// if(logger.isDebugEnabled()) logger.debug(String.format("Running command %s sending to host %s:%d", cmd, host, port));
+ CONNECTION_TYPE connection = connectionHolder.get();
+ if (connection != null) {
+ if (connection.isActive()) {
+ cmd.connectionAvailable(connection);
+// logger.debug("Connection available and active, command run inline.");
+ return;
+ } else {
+ // remove the old connection. (don't worry if we fail since someone else should have done it.
+ connectionHolder.compareAndSet(connection, null);
+ }
+ }
+
+ /**
+ * We've arrived here without a connection, let's make sure only one of us makes a connection. (fyi, another
+ * endpoint could create a reverse connection
+ **/
+ synchronized (this) {
+ connection = connectionHolder.get();
+ if (connection != null) {
+ cmd.connectionAvailable(connection);
+
+ } else {
+// logger.debug("No connection active, opening client connection.");
+ BasicClient<?, CONNECTION_TYPE, OUTBOUND_HANDSHAKE, ?> client = getNewClient();
+ ConnectionListeningFuture<R, C> future = new ConnectionListeningFuture<R, C>(cmd);
+ client.connectAsClient(future, handshake, host, port);
+ future.waitAndRun();
+// logger.debug("Connection available and active, command now being run inline.");
+ }
+ return;
+
+ }
+ }
+
+ public class ConnectionListeningFuture<R extends MessageLite, C extends RpcCommand<R, CONNECTION_TYPE>> extends
+ AbstractFuture<CONNECTION_TYPE> implements RpcConnectionHandler<CONNECTION_TYPE> {
+
+ private C cmd;
+
+ public ConnectionListeningFuture(C cmd) {
+ super();
+ this.cmd = cmd;
+ }
+
+ /**
+ * Called by
+ */
+ public void waitAndRun() {
+ try {
+// logger.debug("Waiting for connection.");
+ CONNECTION_TYPE connection = this.get();
+
+ if (connection == null) {
+// logger.debug("Connection failed.");
+ return;
+ } else {
+// logger.debug("Connection received. {}", connection);
+ cmd.connectionSucceeded(connection);
+// logger.debug("Finished connection succeeded activity.");
+ }
+ } catch (InterruptedException e) {
+ cmd.connectionFailed(FailureType.CONNECTION, e);
+ } catch (ExecutionException e) {
+ throw new IllegalStateException();
+ }
+
+ }
+
+ @Override
+ public void connectionFailed(org.apache.drill.exec.rpc.RpcConnectionHandler.FailureType type, Throwable t) {
+ set(null);
+ cmd.connectionFailed(type, t);
+ }
+
+ @Override
+ public void connectionSucceeded(CONNECTION_TYPE incoming) {
+ CONNECTION_TYPE connection = connectionHolder.get();
+ while (true) {
+ boolean setted = connectionHolder.compareAndSet(null, incoming);
+ if (setted) {
+ connection = incoming;
+ break;
+ }
+ connection = connectionHolder.get();
+ if (connection != null) break;
+ }
+
+ if (connection != incoming) {
+ // close the incoming because another channel was created in the mean time (unless this is a self connection).
+ logger.debug("Closing incoming connection because a connection was already set.");
+ incoming.getChannel().close();
+ }
+ set(connection);
+
+ }
+
+ }
+
+ /** Factory for close handlers **/
+ public class CloseHandlerCreator {
+ public GenericFutureListener<ChannelFuture> getHandler(CONNECTION_TYPE connection,
+ GenericFutureListener<ChannelFuture> parent) {
+ return new CloseHandler(connection, parent);
+ }
+ }
+
+ /**
+ * Listens for connection closes and clears connection holder.
+ */
+ protected class CloseHandler implements GenericFutureListener<ChannelFuture> {
+ private CONNECTION_TYPE connection;
+ private GenericFutureListener<ChannelFuture> parent;
+
+ public CloseHandler(CONNECTION_TYPE connection, GenericFutureListener<ChannelFuture> parent) {
+ super();
+ this.connection = connection;
+ this.parent = parent;
+ }
+
+ @Override
+ public void operationComplete(ChannelFuture future) throws Exception {
+ connectionHolder.compareAndSet(connection, null);
+ parent.operationComplete(future);
+ }
+
+ }
+
+ public CloseHandlerCreator getCloseHandlerCreator() {
+ return new CloseHandlerCreator();
+ }
+
+ public void addExternalConnection(CONNECTION_TYPE connection) {
+ // if the connection holder is not set, set it to this incoming connection. We'll simply ignore if already set.
+ this.connectionHolder.compareAndSet(null, connection);
+ }
+
+ @Override
+ public void close() {
+ CONNECTION_TYPE c = connectionHolder.getAndSet(null);
+ if (c != null) {
+ c.getChannel().close();
+ }
+ }
+
+ /**
+ * Decorate a connection creation so that we capture a success and keep it available for future requests. If we have
+ * raced and another is already available... we return that one and close things down on this one.
+ */
+ private class ConnectionListeningDecorator implements RpcConnectionHandler<CONNECTION_TYPE> {
+
+ private final RpcConnectionHandler<CONNECTION_TYPE> delegate;
+
+ public ConnectionListeningDecorator(RpcConnectionHandler<CONNECTION_TYPE> delegate) {
+ this.delegate = delegate;
+ }
+
+ @Override
+ public void connectionSucceeded(CONNECTION_TYPE incoming) {
+ CONNECTION_TYPE connection = connectionHolder.get();
+ while (true) {
+ boolean setted = connectionHolder.compareAndSet(null, incoming);
+ if (setted) {
+ connection = incoming;
+ break;
+ }
+ connection = connectionHolder.get();
+ if (connection != null)
+ break;
+ }
+
+ if (connection == incoming) {
+ delegate.connectionSucceeded(connection);
+ } else {
+ // close the incoming because another channel was created in the mean time (unless this is a self connection).
+ logger.debug("Closing incoming connection because a connection was already set.");
+ incoming.getChannel().close();
+ delegate.connectionSucceeded(connection);
+ }
+ }
+
+ @Override
+ public void connectionFailed(org.apache.drill.exec.rpc.RpcConnectionHandler.FailureType type, Throwable t) {
+ delegate.connectionFailed(type, t);
+ }
+
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
index 657b350dd..0eaade8f8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
@@ -18,48 +18,88 @@
package org.apache.drill.exec.rpc;
import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
-import java.util.concurrent.Semaphore;
+import org.apache.drill.exec.memory.BufferAllocator;
-public class RemoteConnection{
+public abstract class RemoteConnection{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteConnection.class);
private final Channel channel;
+ private final WriteManager writeManager;
- final Semaphore throttle;
-
- public void acquirePermit() throws InterruptedException{
- if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Acquiring send permit.");
- this.throttle.acquire();
- if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Send permit acquired.");
- }
-
- public void releasePermit() {
- throttle.release();
+ public boolean inEventLoop(){
+ return channel.eventLoop().inEventLoop();
}
- public RemoteConnection(Channel channel, int maxOutstanding) {
+ public RemoteConnection(Channel channel) {
super();
this.channel = channel;
- this.throttle = new Semaphore(maxOutstanding);
+ this.writeManager = new WriteManager();
+ channel.pipeline().addLast(new BackPressureHandler());
}
- public RemoteConnection(Channel channel) {
- this(channel, 100);
- }
-
+ public abstract BufferAllocator getAllocator();
public final Channel getChannel() {
return channel;
}
+ public boolean blockOnNotWritable(RpcOutcomeListener<?> listener){
+ try{
+ writeManager.waitForWritable();
+ return true;
+ }catch(InterruptedException e){
+ listener.failed(new RpcException(e));
+ return false;
+ }
+ }
- public ConnectionThrottle getConnectionThrottle(){
- // can't be implemented until we switch to per query sockets.
- return null;
+ public void setAutoRead(boolean enableAutoRead){
+ channel.config().setAutoRead(enableAutoRead);
}
- public interface ConnectionThrottle{
- public void disableReceiving();
- public void enableReceiving();
+ public boolean isActive(){
+ return channel.isActive();
+ }
+
+ /**
+ * The write manager is responsible for controlling whether or not a write can be sent. It controls whether or not to block a sender if we have tcp backpressure on the receive side.
+ */
+ private static class WriteManager{
+ private final ResettableBarrier barrier = new ResettableBarrier();
+
+ public WriteManager(){
+ barrier.openBarrier();
+ }
+
+ public void waitForWritable() throws InterruptedException{
+ barrier.await();
+ }
+
+ public void setWritable(boolean isWritable){
+// logger.debug("Set writable: {}", isWritable);
+ if(isWritable){
+ barrier.openBarrier();
+ }else{
+ barrier.closeBarrier();
+ }
+
+ }
+
+ }
+
+
+
+ private class BackPressureHandler extends ChannelInboundHandlerAdapter{
+
+ @Override
+ public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
+// logger.debug("Channel writability changed.", ctx.channel().isWritable());
+ writeManager.setWritable(ctx.channel().isWritable());
+ ctx.fireChannelWritabilityChanged();
+ }
+
+
}
} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ResettableBarrier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ResettableBarrier.java
new file mode 100644
index 000000000..a19b75391
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ResettableBarrier.java
@@ -0,0 +1,85 @@
+/**
+ * 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.rpc;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.AbstractQueuedSynchronizer;
+
+/**
+ * Modified implementation of countdown latch that allows a barrier to be unilaterally opened and closed. All others simply wait when it is closed. Is initialized in a closed state.
+ */
+public class ResettableBarrier {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ResettableBarrier.class);
+
+ private final InternalSynchronizer sync = new InternalSynchronizer();
+
+ public ResettableBarrier() {
+ }
+
+ private static final class InternalSynchronizer extends AbstractQueuedSynchronizer {
+
+ private InternalSynchronizer() {
+ setState(1);
+ }
+
+ protected int tryAcquireShared(int acquires) {
+ assert acquires == 1;
+ return (getState() == 0) ? 1 : -1;
+ }
+
+ protected boolean tryReleaseShared(int releases) {
+ assert releases == 1;
+
+ while(true) {
+ int c = getState();
+ if (c == 0)
+ return false;
+ int nextc = c - 1;
+ if (compareAndSetState(c, nextc)){
+ return nextc == 0;
+ }
+ }
+ }
+
+ protected void reset() {
+ setState(1);
+ }
+
+ }
+
+ public void await() throws InterruptedException {
+// logger.debug("awaiting barrier interruptibly.");
+ sync.acquireSharedInterruptibly(1);
+ }
+
+ public boolean await(long timeout, TimeUnit unit) throws InterruptedException {
+// logger.debug("awaiting barrier with timeout {}.", timeout);
+ return sync.tryAcquireSharedNanos(1, unit.toNanos(timeout));
+ }
+
+ public void openBarrier() {
+// logger.debug("opening barrier.");
+ sync.releaseShared(1);
+ }
+
+ public void closeBarrier(){
+// logger.debug("closing barrier.");
+ sync.reset();
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
index b65e7567e..12b73a8ce 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
@@ -17,8 +17,6 @@
*/
package org.apache.drill.exec.rpc;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
-
import io.netty.buffer.ByteBuf;
import com.google.protobuf.Internal.EnumLite;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
index 3f3143110..1be53925b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
@@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufInputStream;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.MessageToMessageDecoder;
import io.netty.util.concurrent.GenericFutureListener;
@@ -61,7 +62,7 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
this.rpcConfig = rpcConfig;
}
- public <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(C connection, T rpcType,
+ <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(C connection, T rpcType,
SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
DrillRpcFutureImpl<RECEIVE> rpcFuture = new DrillRpcFutureImpl<RECEIVE>();
this.send(rpcFuture, connection, rpcType, protobufBody, clazz, dataBodies);
@@ -70,27 +71,35 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> listener, C connection, T rpcType,
SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
+ send(listener, connection, rpcType, protobufBody, clazz, false, dataBodies);
+ }
-
-
-
- assert !Arrays.asList(dataBodies).contains(null);
- assert rpcConfig.checkSend(rpcType, protobufBody.getClass(), clazz);
+ public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> listener, C connection, T rpcType,
+ SEND protobufBody, Class<RECEIVE> clazz, boolean allowInEventLoop, ByteBuf... dataBodies) {
+
+ if(!allowInEventLoop){
+ if(connection.inEventLoop()) throw new IllegalStateException("You attempted to send while inside the rpc event thread. This isn't allowed because sending will block if the channel is backed up.");
+
+ if(!connection.blockOnNotWritable(listener)) return;
+ }
ByteBuf pBuffer = null;
boolean completed = false;
try {
+
+ assert !Arrays.asList(dataBodies).contains(null);
+ assert rpcConfig.checkSend(rpcType, protobufBody.getClass(), clazz);
+
Preconditions.checkNotNull(protobufBody);
ChannelListenerWithCoordinationId futureListener = queue.get(listener, clazz, connection);
OutboundRpcMessage m = new OutboundRpcMessage(RpcMode.REQUEST, rpcType, futureListener.getCoordinationId(), protobufBody, dataBodies);
- connection.acquirePermit();
ChannelFuture channelFuture = connection.getChannel().writeAndFlush(m);
channelFuture.addListener(futureListener);
+ channelFuture.addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
completed = true;
- } catch (InterruptedException e) {
- completed = true;
- listener.failed(new RpcException("Interrupted while attempting to acquire outbound queue.", e));
+ } catch(Exception | AssertionError e){
+ listener.failed(new RpcException("Failure sending message.", e));
} finally {
if (!completed) {
if (pBuffer != null) pBuffer.release();
@@ -167,6 +176,7 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
case RESPONSE_FAILURE:
RpcFailure failure = RpcFailure.parseFrom(new ByteBufInputStream(msg.pBody, msg.pBody.readableBytes()));
queue.updateFailedFuture(msg.coordinationId, failure);
+ msg.release();
if (RpcConstants.EXTRA_DEBUGGING)
logger.debug("Updated rpc future with coordinationId {} with failure ", msg.coordinationId, failure);
break;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCommand.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcCommand.java
index 90cbf932e..952317771 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCommand.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcCommand.java
@@ -15,14 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc;
import org.apache.drill.exec.rpc.RpcConnectionHandler;
import com.google.protobuf.MessageLite;
-public interface BitCommand<T extends MessageLite> extends RpcConnectionHandler<BitConnection>{
+public interface RpcCommand<T extends MessageLite, C extends RemoteConnection> extends RpcConnectionHandler<C>{
- public abstract void connectionAvailable(BitConnection connection);
+ public abstract void connectionAvailable(C connection);
} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
index d0a8086ad..42f733b6b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
@@ -29,9 +29,14 @@ public class RpcExceptionHandler implements ChannelHandler{
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
- if(!ctx.channel().isOpen()) return;
- logger.error("Exception in pipeline. Closing channel between local " + ctx.channel().localAddress() + " and remote " + ctx.channel().remoteAddress(), cause);
- ctx.close();
+
+ if(!ctx.channel().isOpen()){
+ logger.warn("Exception with closed channel", cause);
+ return;
+ }else{
+ logger.error("Exception in pipeline. Closing channel between local " + ctx.channel().localAddress() + " and remote " + ctx.channel().remoteAddress(), cause);
+ ctx.close();
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
deleted file mode 100644
index 7ada32c8f..000000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
+++ /dev/null
@@ -1,99 +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.rpc.bit;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.util.concurrent.GenericFutureListener;
-
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
-import org.apache.drill.exec.rpc.BasicClient;
-import org.apache.drill.exec.rpc.Response;
-import org.apache.drill.exec.rpc.RpcBus;
-import org.apache.drill.exec.rpc.RpcConnectionHandler;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitConnectionManager.CloseHandlerCreator;
-import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.work.batch.BitComHandler;
-
-import com.google.protobuf.MessageLite;
-
-public class BitClient extends BasicClient<RpcType, BitConnection, BitHandshake, BitHandshake>{
-
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitClient.class);
-
- private final BitComHandler handler;
- private final DrillbitEndpoint remoteEndpoint;
- private volatile BitConnection connection;
- private final ListenerPool listeners;
- private final CloseHandlerCreator closeHandlerFactory;
- private final DrillbitEndpoint localIdentity;
-
- public BitClient(DrillbitEndpoint remoteEndpoint, DrillbitEndpoint localEndpoint, BitComHandler handler, BootStrapContext context, CloseHandlerCreator closeHandlerFactory, ListenerPool listeners) {
- super(BitRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup(), RpcType.HANDSHAKE, BitHandshake.class, BitHandshake.PARSER);
- this.localIdentity = localEndpoint;
- this.remoteEndpoint = remoteEndpoint;
- this.handler = handler;
- this.listeners = listeners;
- this.closeHandlerFactory = closeHandlerFactory;
- }
-
- public void connect(RpcConnectionHandler<BitConnection> connectionHandler) {
- connectAsClient(connectionHandler, BitHandshake.newBuilder().setRpcVersion(BitRpcConfig.RPC_VERSION).setEndpoint(localIdentity).build(), remoteEndpoint.getAddress(), remoteEndpoint.getBitPort());
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public BitConnection initRemoteConnection(Channel channel) {
- this.connection = new BitConnection(channel, (RpcBus<RpcType, BitConnection>) (RpcBus<?, ?>) this, listeners);
- return connection;
- }
-
- @Override
- protected GenericFutureListener<ChannelFuture> getCloseHandler(BitConnection clientConnection) {
- return closeHandlerFactory.getHandler(clientConnection, super.getCloseHandler(clientConnection));
- }
-
- @Override
- public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
- return BitComDefaultInstanceHandler.getResponseDefaultInstance(rpcType);
- }
-
- @Override
- protected Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(connection, rpcType, pBody, dBody);
- }
-
- @Override
- protected void validateHandshake(BitHandshake handshake) throws RpcException {
- if(handshake.getRpcVersion() != BitRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", handshake.getRpcVersion(), BitRpcConfig.RPC_VERSION));
- }
-
- @Override
- protected void finalizeConnection(BitHandshake handshake, BitConnection connection) {
- connection.setEndpoint(handshake.getEndpoint());
- }
-
- public BitConnection getConnection(){
- return this.connection;
- }
-
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
deleted file mode 100644
index 6c6b6a06d..000000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
+++ /dev/null
@@ -1,189 +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.rpc.bit;
-
-import io.netty.channel.ChannelFuture;
-import io.netty.util.concurrent.GenericFutureListener;
-
-import java.io.Closeable;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.rpc.RpcConnectionHandler;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.work.batch.BitComHandler;
-
-import com.google.protobuf.MessageLite;
-
-/**
- * Manager all connections between two particular bits.
- */
-public class BitConnectionManager implements Closeable{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitConnectionManager.class);
-
- private final DrillbitEndpoint endpoint;
- private final AtomicReference<BitConnection> connectionHolder;
- private final BitComHandler handler;
- private final BootStrapContext context;
- private final ListenerPool listenerPool;
- private final DrillbitEndpoint localIdentity;
-
- public BitConnectionManager(DrillbitEndpoint remoteEndpoint, DrillbitEndpoint localIdentity, BitComHandler handler, BootStrapContext context, ListenerPool listenerPool) {
- assert remoteEndpoint != null : "Endpoint cannot be null.";
- assert remoteEndpoint.getAddress() != null && !remoteEndpoint.getAddress().isEmpty(): "Endpoint address cannot be null.";
- assert remoteEndpoint.getBitPort() > 0 : String.format("Bit Port must be set to a port between 1 and 65k. Was set to %d.", remoteEndpoint.getBitPort());
-
- this.connectionHolder = new AtomicReference<BitConnection>();
- this.endpoint = remoteEndpoint;
- this.localIdentity = localIdentity;
- this.handler = handler;
- this.context = context;
- this.listenerPool = listenerPool;
- }
-
- public <R extends MessageLite> BitCommand<R> runCommand(BitCommand<R> cmd){
- logger.debug("Running command {}", cmd);
- BitConnection connection = connectionHolder.get();
- if(connection != null){
- if(connection.isActive()){
- cmd.connectionAvailable(connection);
- return cmd;
- }else{
- // remove the old connection. (don't worry if we fail since someone else should have done it.
- connectionHolder.compareAndSet(connection, null);
- }
- }
-
- /** We've arrived here without a connection, let's make sure only one of us makes a connection. (fyi, another endpoint could create a reverse connection **/
- synchronized(this){
- connection = connectionHolder.get();
- if(connection != null){
- cmd.connectionAvailable(connection);
- }else{
- BitClient client = new BitClient(endpoint, localIdentity, handler, context, new CloseHandlerCreator(), listenerPool);
-
- client.connect(new ConnectionListeningDecorator(cmd, !endpoint.equals(localIdentity)));
- }
- return cmd;
-
- }
- }
-
- CloseHandlerCreator getCloseHandlerCreator(){
- return new CloseHandlerCreator();
- }
-
- /** Factory for close handlers **/
- class CloseHandlerCreator{
- public GenericFutureListener<ChannelFuture> getHandler(BitConnection connection, GenericFutureListener<ChannelFuture> parent){
- return new CloseHandler(connection, parent);
- }
- }
-
-
-
- /**
- * Listens for connection closes and clears connection holder.
- */
- private class CloseHandler implements GenericFutureListener<ChannelFuture>{
- private BitConnection connection;
- private GenericFutureListener<ChannelFuture> parent;
-
- public CloseHandler(BitConnection connection, GenericFutureListener<ChannelFuture> parent) {
- super();
- this.connection = connection;
- this.parent = parent;
- }
-
- @Override
- public void operationComplete(ChannelFuture future) throws Exception {
- connectionHolder.compareAndSet(connection, null);
- parent.operationComplete(future);
- }
-
- }
-
- /**
- * Decorate a connection creation so that we capture a success and keep it available for future requests. If we have raced and another is already available... we return that one and close things down on this one.
- */
- private class ConnectionListeningDecorator implements RpcConnectionHandler<BitConnection>{
-
- private final RpcConnectionHandler<BitConnection> delegate;
- private final boolean closeOnDupe;
-
- public ConnectionListeningDecorator(RpcConnectionHandler<BitConnection> delegate, boolean closeOnDupe) {
- this.delegate = delegate;
- this.closeOnDupe = closeOnDupe;
- }
-
- @Override
- public void connectionSucceeded(BitConnection incoming) {
- BitConnection connection = connectionHolder.get();
- while(true){
- boolean setted = connectionHolder.compareAndSet(null, incoming);
- if(setted){
- connection = incoming;
- break;
- }
- connection = connectionHolder.get();
- if(connection != null) break;
- }
-
-
- if(connection == incoming){
- delegate.connectionSucceeded(connection);
- }else{
-
- if(closeOnDupe){
- // close the incoming because another channel was created in the mean time (unless this is a self connection).
- logger.debug("Closing incoming connection because a connection was already set.");
- incoming.getChannel().close();
- }
- delegate.connectionSucceeded(connection);
- }
- }
-
- @Override
- public void connectionFailed(org.apache.drill.exec.rpc.RpcConnectionHandler.FailureType type, Throwable t) {
- delegate.connectionFailed(type, t);
- }
-
- }
-
- public DrillbitEndpoint getEndpoint() {
- return endpoint;
- }
-
- public void addServerConnection(BitConnection connection){
- // if the connection holder is not set, set it to this incoming connection.
- logger.debug("Setting server connection.");
- this.connectionHolder.compareAndSet(null, connection);
- }
-
- @Override
- public void close() {
- BitConnection c = connectionHolder.getAndSet(null);
- if(c != null){
- c.getChannel().close();
- }
- }
-
-
-
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
deleted file mode 100644
index 1f85625b1..000000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
+++ /dev/null
@@ -1,59 +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.rpc.bit;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.work.foreman.FragmentStatusListener;
-
-public class ListenerPool {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ListenerPool.class);
-
- private final ConcurrentMap<QueryId, FragmentStatusListener> listeners;
-
- public ListenerPool(int par){
- listeners = new ConcurrentHashMap<QueryId, FragmentStatusListener>(16, 0.75f, par);
- }
-
- public void removeFragmentStatusListener(FragmentHandle handle) throws RpcException{
- logger.debug("Removing framgent status listener for handle {}.", handle);
- listeners.remove(handle);
- }
-
- public void addFragmentStatusListener(FragmentHandle handle, FragmentStatusListener listener) throws RpcException{
- logger.debug("Adding framgent status listener for handle {}.", handle);
- FragmentStatusListener old = listeners.putIfAbsent(handle.getQueryId(), listener);
- if(old != null) throw new RpcException("Failure. The provided handle already exists in the listener pool. You need to remove one listener before adding another.");
- }
-
- public void status(FragmentStatus status){
- FragmentStatusListener l = listeners.get(status.getHandle().getQueryId());
- if(l == null){
-
- logger.error("A fragment message arrived but there was no registered listener for that message for handle {}.", status.getHandle());
- return;
- }else{
- l.statusUpdate(status);
- }
- }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/AvailabilityListener.java
index ff3a03207..4f817c668 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/AvailabilityListener.java
@@ -15,8 +15,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc.control;
public interface AvailabilityListener {
- public void isAvailable(BitConnection connection);
+ public void isAvailable(ControlConnection connection);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ConnectionManagerRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ConnectionManagerRegistry.java
index af8fd9cec..649dc09ff 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ConnectionManagerRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ConnectionManagerRegistry.java
@@ -15,46 +15,38 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
-
-import io.netty.channel.Channel;
+package org.apache.drill.exec.rpc.control;
import java.util.Iterator;
import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicReference;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
-import com.google.common.base.Preconditions;
import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.CheckedFuture;
-public class ConnectionManagerRegistry implements Iterable<BitConnectionManager>{
+public class ConnectionManagerRegistry implements Iterable<ControlConnectionManager>{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ConnectionManagerRegistry.class);
- private final ConcurrentMap<DrillbitEndpoint, BitConnectionManager> registry = Maps.newConcurrentMap();
+ private final ConcurrentMap<DrillbitEndpoint, ControlConnectionManager> registry = Maps.newConcurrentMap();
- private final BitComHandler handler;
+ private final ControlMessageHandler handler;
private final BootStrapContext context;
- private final ListenerPool listenerPool;
private volatile DrillbitEndpoint localEndpoint;
- public ConnectionManagerRegistry(BitComHandler handler, BootStrapContext context, ListenerPool listenerPool) {
+ public ConnectionManagerRegistry(ControlMessageHandler handler, BootStrapContext context) {
super();
this.handler = handler;
this.context = context;
- this.listenerPool = listenerPool;
}
- public BitConnectionManager getConnectionManager(DrillbitEndpoint endpoint){
+ public ControlConnectionManager getConnectionManager(DrillbitEndpoint endpoint){
assert localEndpoint != null : "DrillbitEndpoint must be set before a connection manager can be retrieved";
- BitConnectionManager m = registry.get(endpoint);
+ ControlConnectionManager m = registry.get(endpoint);
if(m == null){
- m = new BitConnectionManager(endpoint, localEndpoint, handler, context, listenerPool);
- BitConnectionManager m2 = registry.putIfAbsent(endpoint, m);
+ m = new ControlConnectionManager(endpoint, localEndpoint, handler, context);
+ ControlConnectionManager m2 = registry.putIfAbsent(endpoint, m);
if(m2 != null) m = m2;
}
@@ -62,7 +54,7 @@ public class ConnectionManagerRegistry implements Iterable<BitConnectionManager>
}
@Override
- public Iterator<BitConnectionManager> iterator() {
+ public Iterator<ControlConnectionManager> iterator() {
return registry.values().iterator();
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java
new file mode 100644
index 000000000..af2b58bf5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java
@@ -0,0 +1,105 @@
+/**
+ * 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.rpc.control;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitControl.BitControlHandshake;
+import org.apache.drill.exec.proto.BitControl.RpcType;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.BasicClient;
+import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcBus;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
+
+import com.google.protobuf.MessageLite;
+
+public class ControlClient extends BasicClient<RpcType, ControlConnection, BitControlHandshake, BitControlHandshake>{
+
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlClient.class);
+
+ private final ControlMessageHandler handler;
+ private final DrillbitEndpoint remoteEndpoint;
+ private volatile ControlConnection connection;
+ private final ControlConnectionManager.CloseHandlerCreator closeHandlerFactory;
+ private final DrillbitEndpoint localIdentity;
+ private final BufferAllocator allocator;
+
+ public ControlClient(DrillbitEndpoint remoteEndpoint, DrillbitEndpoint localEndpoint, ControlMessageHandler handler, BootStrapContext context, ControlConnectionManager.CloseHandlerCreator closeHandlerFactory) {
+ super(ControlRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup(), RpcType.HANDSHAKE, BitControlHandshake.class, BitControlHandshake.PARSER);
+ this.localIdentity = localEndpoint;
+ this.remoteEndpoint = remoteEndpoint;
+ this.handler = handler;
+ this.closeHandlerFactory = closeHandlerFactory;
+ this.allocator = context.getAllocator();
+ }
+
+ public void connect(RpcConnectionHandler<ControlConnection> connectionHandler) {
+ connectAsClient(connectionHandler, BitControlHandshake.newBuilder().setRpcVersion(ControlRpcConfig.RPC_VERSION).setEndpoint(localIdentity).build(), remoteEndpoint.getAddress(), remoteEndpoint.getControlPort());
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public ControlConnection initRemoteConnection(Channel channel) {
+ this.connection = new ControlConnection(channel, (RpcBus<RpcType, ControlConnection>) (RpcBus<?, ?>) this, allocator);
+ return connection;
+ }
+
+ @Override
+ protected GenericFutureListener<ChannelFuture> getCloseHandler(ControlConnection clientConnection) {
+ return closeHandlerFactory.getHandler(clientConnection, super.getCloseHandler(clientConnection));
+ }
+
+ @Override
+ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+ return DefaultInstanceHandler.getResponseDefaultInstance(rpcType);
+ }
+
+ @Override
+ protected Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ return handler.handle(connection, rpcType, pBody, dBody);
+ }
+
+ @Override
+ protected void validateHandshake(BitControlHandshake handshake) throws RpcException {
+ if(handshake.getRpcVersion() != ControlRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+ }
+
+ @Override
+ protected void finalizeConnection(BitControlHandshake handshake, ControlConnection connection) {
+ connection.setEndpoint(handshake.getEndpoint());
+ }
+
+ public ControlConnection getConnection(){
+ return this.connection;
+ }
+
+ @Override
+ public ProtobufLengthDecoder getDecoder(BufferAllocator allocator) {
+ return new ControlProtobufLengthDecoder(allocator);
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlCommand.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlCommand.java
new file mode 100644
index 000000000..52d42896c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlCommand.java
@@ -0,0 +1,28 @@
+/**
+ * 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.rpc.control;
+
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
+
+import com.google.protobuf.MessageLite;
+
+public interface ControlCommand<T extends MessageLite> extends RpcConnectionHandler<ControlConnection>{
+
+ public abstract void connectionAvailable(ControlConnection connection);
+
+} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnection.java
index 97b5a7936..af0368a28 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnection.java
@@ -15,52 +15,40 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc.control;
import io.netty.buffer.ByteBuf;
import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.util.concurrent.GenericFutureListener;
import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitControl.RpcType;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.FragmentWritableBatch;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
import org.apache.drill.exec.rpc.RemoteConnection;
import org.apache.drill.exec.rpc.RpcBus;
-import org.apache.drill.exec.rpc.RpcConnectionHandler;
import org.apache.drill.exec.rpc.RpcOutcomeListener;
-import com.google.common.base.Preconditions;
import com.google.common.io.Closeables;
import com.google.protobuf.MessageLite;
-public class BitConnection extends RemoteConnection{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitConnection.class);
-
- private final RpcBus<RpcType, BitConnection> bus;
- private final ListenerPool listeners;
+public class ControlConnection extends RemoteConnection {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlConnection.class);
+
+ private final RpcBus<RpcType, ControlConnection> bus;
+ private final BufferAllocator allocator;
private volatile DrillbitEndpoint endpoint;
private volatile boolean active = false;
private final UUID id;
-
- public BitConnection(Channel channel, RpcBus<RpcType, BitConnection> bus, ListenerPool listeners){
+
+ public ControlConnection(Channel channel, RpcBus<RpcType, ControlConnection> bus, BufferAllocator allocator) {
super(channel);
this.bus = bus;
- // we use a local listener pool unless a global one is provided.
- this.listeners = listeners != null ? listeners : new ListenerPool(2);
this.id = UUID.randomUUID();
+ this.allocator = allocator;
}
-
- void setEndpoint(DrillbitEndpoint endpoint){
+
+ void setEndpoint(DrillbitEndpoint endpoint) {
assert this.endpoint == null : "Endpoint should only be set once (only in the case in incoming server requests).";
this.endpoint = endpoint;
active = true;
@@ -70,25 +58,19 @@ public class BitConnection extends RemoteConnection{
return endpoint;
}
- public ListenerPool getListenerPool(){
- return listeners;
- }
-
-
- public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> outcomeListener, RpcType rpcType,
- SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies){
+ public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> outcomeListener,
+ RpcType rpcType, SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
bus.send(outcomeListener, this, rpcType, protobufBody, clazz, dataBodies);
}
-
- public void disable(){
+ public void disable() {
active = false;
}
-
- public boolean isActive(){
+
+ public boolean isActive() {
return active;
}
-
+
@Override
public int hashCode() {
final int prime = 31;
@@ -99,19 +81,28 @@ public class BitConnection extends RemoteConnection{
@Override
public boolean equals(Object obj) {
- if (this == obj) return true;
- if (obj == null) return false;
- if (getClass() != obj.getClass()) return false;
- BitConnection other = (BitConnection) obj;
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ ControlConnection other = (ControlConnection) obj;
if (id == null) {
- if (other.id != null) return false;
- } else if (!id.equals(other.id)) return false;
+ if (other.id != null)
+ return false;
+ } else if (!id.equals(other.id))
+ return false;
return true;
}
+ public void shutdownIfClient() {
+ if (bus.isClient())
+ Closeables.closeQuietly(bus);
+ }
-
- public void shutdownIfClient(){
- if(bus.isClient()) Closeables.closeQuietly(bus);
+ @Override
+ public BufferAllocator getAllocator() {
+ return allocator;
}
} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnectionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnectionManager.java
new file mode 100644
index 000000000..f3274a973
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnectionManager.java
@@ -0,0 +1,60 @@
+/**
+ * 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.rpc.control;
+
+import org.apache.drill.exec.proto.BitControl.BitControlHandshake;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.BasicClient;
+import org.apache.drill.exec.rpc.ReconnectingConnection;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
+
+/**
+ * Maintains connection between two particular bits.
+ */
+public class ControlConnectionManager extends ReconnectingConnection<ControlConnection, BitControlHandshake>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlConnectionManager.class);
+
+ private final DrillbitEndpoint endpoint;
+ private final ControlMessageHandler handler;
+ private final BootStrapContext context;
+ private final DrillbitEndpoint localIdentity;
+
+ public ControlConnectionManager(DrillbitEndpoint remoteEndpoint, DrillbitEndpoint localIdentity, ControlMessageHandler handler, BootStrapContext context) {
+ super(BitControlHandshake.newBuilder().setRpcVersion(ControlRpcConfig.RPC_VERSION).setEndpoint(localIdentity).build(), remoteEndpoint.getAddress(), remoteEndpoint.getControlPort());
+ assert remoteEndpoint != null : "Endpoint cannot be null.";
+ assert remoteEndpoint.getAddress() != null && !remoteEndpoint.getAddress().isEmpty(): "Endpoint address cannot be null.";
+ assert remoteEndpoint.getControlPort() > 0 : String.format("Bit Port must be set to a port between 1 and 65k. Was set to %d.", remoteEndpoint.getControlPort());
+
+ this.endpoint = remoteEndpoint;
+ this.localIdentity = localIdentity;
+ this.handler = handler;
+ this.context = context;
+ }
+
+ @Override
+ protected BasicClient<?, ControlConnection, BitControlHandshake, ?> getNewClient() {
+ return new ControlClient(endpoint, localIdentity, handler, context, new CloseHandlerCreator());
+ }
+
+
+ public DrillbitEndpoint getEndpoint() {
+ return endpoint;
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlProtobufLengthDecoder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlProtobufLengthDecoder.java
new file mode 100644
index 000000000..c00dc5483
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlProtobufLengthDecoder.java
@@ -0,0 +1,40 @@
+/**
+ * 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.rpc.control;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+
+import java.util.List;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
+
+/**
+ * Purely to simplify memory debugging.
+ */
+public class ControlProtobufLengthDecoder extends ProtobufLengthDecoder{
+ public ControlProtobufLengthDecoder(BufferAllocator allocator) {
+ super(allocator);
+
+ }
+ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
+ super.decode(ctx, in, out);
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlRpcConfig.java
index 8551357ba..53e5d1a85 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlRpcConfig.java
@@ -15,29 +15,27 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc.control;
-import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.BitControl.BitControlHandshake;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.RpcType;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.rpc.Acks;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.RpcConfig;
-public class BitRpcConfig {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitRpcConfig.class);
+public class ControlRpcConfig {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlRpcConfig.class);
- public static RpcConfig MAPPING = RpcConfig.newBuilder("BIT-RPC-MAPPING") //
- .add(RpcType.HANDSHAKE, BitHandshake.class, RpcType.HANDSHAKE, BitHandshake.class)
+ public static RpcConfig MAPPING = RpcConfig.newBuilder("BIT-CONTROL-RPC-MAPPING") //
+ .add(RpcType.HANDSHAKE, BitControlHandshake.class, RpcType.HANDSHAKE, BitControlHandshake.class)
.add(RpcType.REQ_INIATILIZE_FRAGMENT, PlanFragment.class, RpcType.ACK, Ack.class)
.add(RpcType.REQ_CANCEL_FRAGMENT, FragmentHandle.class, RpcType.ACK, Ack.class)
.add(RpcType.REQ_FRAGMENT_STATUS, FragmentStatus.class, RpcType.ACK, Ack.class)
- .add(RpcType.REQ_RECORD_BATCH, FragmentRecordBatch.class, RpcType.ACK, Ack.class)
.build();
public static int RPC_VERSION = 2;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java
index 049418c6e..4b6a85d4e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java
@@ -15,88 +15,92 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc.control;
import io.netty.buffer.ByteBuf;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.util.concurrent.GenericFutureListener;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitControl.BitControlHandshake;
+import org.apache.drill.exec.proto.BitControl.RpcType;
import org.apache.drill.exec.rpc.BasicServer;
+import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitConnectionManager.CloseHandlerCreator;
import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
import com.google.protobuf.MessageLite;
-public class BitServer extends BasicServer<RpcType, BitConnection>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitServer.class);
+public class ControlServer extends BasicServer<RpcType, ControlConnection>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlServer.class);
- private final BitComHandler handler;
- private final ListenerPool listeners;
+ private final ControlMessageHandler handler;
private final ConnectionManagerRegistry connectionRegistry;
private volatile ProxyCloseHandler proxyCloseHandler;
+ private BufferAllocator allocator;
- public BitServer(BitComHandler handler, BootStrapContext context, ConnectionManagerRegistry connectionRegistry, ListenerPool listeners) {
- super(BitRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup());
+ public ControlServer(ControlMessageHandler handler, BootStrapContext context, ConnectionManagerRegistry connectionRegistry) {
+ super(ControlRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup());
this.handler = handler;
this.connectionRegistry = connectionRegistry;
- this.listeners = listeners;
+ this.allocator = context.getAllocator();
}
@Override
public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
- return BitComDefaultInstanceHandler.getResponseDefaultInstance(rpcType);
+ return DefaultInstanceHandler.getResponseDefaultInstance(rpcType);
}
@Override
- protected Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ protected Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
return handler.handle(connection, rpcType, pBody, dBody);
}
@Override
- protected GenericFutureListener<ChannelFuture> getCloseHandler(BitConnection connection) {
+ protected GenericFutureListener<ChannelFuture> getCloseHandler(ControlConnection connection) {
this.proxyCloseHandler = new ProxyCloseHandler(super.getCloseHandler(connection));
return proxyCloseHandler;
}
@Override
- public BitConnection initRemoteConnection(Channel channel) {
- return new BitConnection(channel, this, listeners);
+ public ControlConnection initRemoteConnection(Channel channel) {
+ return new ControlConnection(channel, this, allocator);
}
@Override
- protected ServerHandshakeHandler<BitHandshake> getHandshakeHandler(final BitConnection connection) {
- return new ServerHandshakeHandler<BitHandshake>(RpcType.HANDSHAKE, BitHandshake.PARSER){
+ protected ServerHandshakeHandler<BitControlHandshake> getHandshakeHandler(final ControlConnection connection) {
+ return new ServerHandshakeHandler<BitControlHandshake>(RpcType.HANDSHAKE, BitControlHandshake.PARSER){
@Override
- public MessageLite getHandshakeResponse(BitHandshake inbound) throws Exception {
+ public MessageLite getHandshakeResponse(BitControlHandshake inbound) throws Exception {
// logger.debug("Handling handshake from other bit. {}", inbound);
- if(inbound.getRpcVersion() != BitRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", inbound.getRpcVersion(), BitRpcConfig.RPC_VERSION));
- if(!inbound.hasEndpoint() || inbound.getEndpoint().getAddress().isEmpty() || inbound.getEndpoint().getBitPort() < 1) throw new RpcException(String.format("RPC didn't provide valid counter endpoint information. Received %s.", inbound.getEndpoint()));
+ if(inbound.getRpcVersion() != ControlRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", inbound.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+ if(!inbound.hasEndpoint() || inbound.getEndpoint().getAddress().isEmpty() || inbound.getEndpoint().getControlPort() < 1) throw new RpcException(String.format("RPC didn't provide valid counter endpoint information. Received %s.", inbound.getEndpoint()));
connection.setEndpoint(inbound.getEndpoint());
// add the
- BitConnectionManager manager = connectionRegistry.getConnectionManager(inbound.getEndpoint());
+ ControlConnectionManager manager = connectionRegistry.getConnectionManager(inbound.getEndpoint());
// update the close handler.
proxyCloseHandler.setHandler(manager.getCloseHandlerCreator().getHandler(connection, proxyCloseHandler.getHandler()));
// add to the connection manager.
- manager.addServerConnection(connection);
+ manager.addExternalConnection(connection);
- return BitHandshake.newBuilder().setRpcVersion(BitRpcConfig.RPC_VERSION).build();
+ return BitControlHandshake.newBuilder().setRpcVersion(ControlRpcConfig.RPC_VERSION).build();
}
};
}
+ @Override
+ public ProtobufLengthDecoder getDecoder(BufferAllocator allocator) {
+ return new ControlProtobufLengthDecoder(allocator);
+ }
private class ProxyCloseHandler implements GenericFutureListener<ChannelFuture> {
@@ -124,4 +128,5 @@ public class BitServer extends BasicServer<RpcType, BitConnection>{
}
+
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java
index bbeb39d88..ea680d9de 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java
@@ -15,28 +15,27 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc.control;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.RpcType;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
-import org.apache.drill.exec.proto.GeneralRPCProtos;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.FragmentWritableBatch;
import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.rpc.FutureBitCommand;
+import org.apache.drill.exec.rpc.ListeningCommand;
import org.apache.drill.exec.rpc.RpcOutcomeListener;
-public class BitTunnel {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitTunnel.class);
+public class ControlTunnel {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlTunnel.class);
- private final BitConnectionManager manager;
+ private final ControlConnectionManager manager;
private final DrillbitEndpoint endpoint;
- public BitTunnel(DrillbitEndpoint endpoint, BitConnectionManager manager) {
+ public ControlTunnel(DrillbitEndpoint endpoint, ControlConnectionManager manager) {
this.manager = manager;
this.endpoint = endpoint;
}
@@ -45,17 +44,6 @@ public class BitTunnel {
return manager.getEndpoint();
}
- public void sendRecordBatch(RpcOutcomeListener<Ack> outcomeListener, FragmentContext context, FragmentWritableBatch batch) {
- SendBatch b = new SendBatch(outcomeListener, batch, context);
- manager.runCommand(b);
- }
-
- public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, FragmentWritableBatch batch) {
- SendBatchAsync b = new SendBatchAsync(batch, context);
- manager.runCommand(b);
- return b.getFuture();
- }
-
public void sendFragment(RpcOutcomeListener<Ack> outcomeListener, PlanFragment fragment){
SendFragment b = new SendFragment(outcomeListener, fragment);
manager.runCommand(b);
@@ -73,51 +61,8 @@ public class BitTunnel {
return b.getFuture();
}
- public static class SendBatchAsync extends FutureBitCommand<Ack> {
- final FragmentWritableBatch batch;
- final FragmentContext context;
-
- public SendBatchAsync(FragmentWritableBatch batch, FragmentContext context) {
- super();
- this.batch = batch;
- this.context = context;
- }
-
- @Override
- public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, BitConnection connection) {
- connection.send(outcomeListener, RpcType.REQ_RECORD_BATCH, batch.getHeader(), Ack.class, batch.getBuffers());
- }
-
- @Override
- public String toString() {
- return "SendBatch [batch.header=" + batch.getHeader() + "]";
- }
- }
-
- public static class SendBatch extends ListeningBitCommand<Ack> {
- final FragmentWritableBatch batch;
- final FragmentContext context;
-
- public SendBatch(RpcOutcomeListener<Ack> listener, FragmentWritableBatch batch, FragmentContext context) {
- super(listener);
- this.batch = batch;
- this.context = context;
- }
-
- @Override
- public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, BitConnection connection) {
- connection.send(outcomeListener, RpcType.REQ_RECORD_BATCH, batch.getHeader(), Ack.class, batch.getBuffers());
- }
-
- @Override
- public String toString() {
- return "SendBatch [batch.header=" + batch.getHeader() + "]";
- }
-
-
- }
- public static class SendFragmentStatus extends FutureBitCommand<Ack> {
+ public static class SendFragmentStatus extends FutureBitCommand<Ack, ControlConnection> {
final FragmentStatus status;
public SendFragmentStatus(FragmentStatus status) {
@@ -126,13 +71,13 @@ public class BitTunnel {
}
@Override
- public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, BitConnection connection) {
+ public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, ControlConnection connection) {
connection.send(outcomeListener, RpcType.REQ_FRAGMENT_STATUS, status, Ack.class);
}
}
- public static class CancelFragment extends FutureBitCommand<Ack> {
+ public static class CancelFragment extends FutureBitCommand<Ack, ControlConnection> {
final FragmentHandle handle;
public CancelFragment(FragmentHandle handle) {
@@ -141,13 +86,13 @@ public class BitTunnel {
}
@Override
- public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, BitConnection connection) {
+ public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, ControlConnection connection) {
connection.send(outcomeListener, RpcType.REQ_CANCEL_FRAGMENT, handle, Ack.class);
}
}
- public static class SendFragment extends ListeningBitCommand<Ack> {
+ public static class SendFragment extends ListeningCommand<Ack, ControlConnection> {
final PlanFragment fragment;
public SendFragment(RpcOutcomeListener<Ack> listener, PlanFragment fragment) {
@@ -156,7 +101,7 @@ public class BitTunnel {
}
@Override
- public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, BitConnection connection) {
+ public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, ControlConnection connection) {
connection.send(outcomeListener, RpcType.REQ_INIATILIZE_FRAGMENT, fragment, Ack.class);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/Controller.java
index b3bff9f10..9a11c9484 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/Controller.java
@@ -15,21 +15,21 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc.control;
import java.io.Closeable;
import org.apache.drill.exec.exception.DrillbitStartupException;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.apache.drill.exec.work.fragment.FragmentManager;
/**
* Service that allows one Drillbit to communicate with another. Internally manages whether each particular bit is a
- * server or a client depending on who initially made the connection. If no connection exists, BitCom is responsible for
- * making a connection. BitCom should automatically straight route local BitCommunication rather than connecting to its
+ * server or a client depending on who initially made the connection. If no connection exists, the Controller is responsible for
+ * making a connection. TODO: Controller should automatically straight route local BitCommunication rather than connecting to its
* self.
*/
-public interface BitCom extends Closeable {
+public interface Controller extends Closeable {
/**
* Get a Bit to Bit communication tunnel. If the BitCom doesn't have a tunnel attached to the node already, it will
@@ -38,19 +38,9 @@ public interface BitCom extends Closeable {
* @param node
* @return
*/
- public BitTunnel getTunnel(DrillbitEndpoint node) ;
+ public ControlTunnel getTunnel(DrillbitEndpoint node) ;
public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws InterruptedException, DrillbitStartupException;
- /**
- * Register an incoming batch handler for a local foreman.
- * @param handler
- */
- public void registerIncomingBatchHandler(IncomingFragmentHandler handler);
- /**
- * Get ListenerPool
- * @return
- */
- public ListenerPool getListeners();
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java
index 0078b07dc..de8caf6ec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc.control;
import java.util.concurrent.ConcurrentMap;
@@ -23,8 +23,8 @@ import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.exception.DrillbitStartupException;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.work.batch.BitComHandler;
-import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
+import org.apache.drill.exec.work.fragment.FragmentManager;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
@@ -32,52 +32,39 @@ import com.google.common.io.Closeables;
/**
* Manages communication tunnels between nodes.
*/
-public class BitComImpl implements BitCom {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComImpl.class);
+public class ControllerImpl implements Controller {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControllerImpl.class);
- private final ListenerPool listeners;
- private volatile BitServer server;
- private final BitComHandler handler;
+ private volatile ControlServer server;
+ private final ControlMessageHandler handler;
private final BootStrapContext context;
private final ConnectionManagerRegistry connectionRegistry;
- public BitComImpl(BootStrapContext context, BitComHandler handler) {
+ public ControllerImpl(BootStrapContext context, ControlMessageHandler handler) {
super();
this.handler = handler;
this.context = context;
- this.listeners = new ListenerPool(8);
- this.connectionRegistry = new ConnectionManagerRegistry(handler, context, listeners);
+ this.connectionRegistry = new ConnectionManagerRegistry(handler, context);
}
@Override
public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws InterruptedException, DrillbitStartupException {
- server = new BitServer(handler, context, connectionRegistry, listeners);
+ server = new ControlServer(handler, context, connectionRegistry);
int port = context.getConfig().getInt(ExecConstants.INITIAL_BIT_PORT);
port = server.bind(port);
- DrillbitEndpoint completeEndpoint = partialEndpoint.toBuilder().setBitPort(port).build();
+ DrillbitEndpoint completeEndpoint = partialEndpoint.toBuilder().setControlPort(port).build();
connectionRegistry.setEndpoint(completeEndpoint);
return completeEndpoint;
}
-
-
- public ListenerPool getListeners() {
- return listeners;
- }
-
- @Override
- public BitTunnel getTunnel(DrillbitEndpoint endpoint) {
- return new BitTunnel(endpoint, connectionRegistry.getConnectionManager(endpoint));
- }
-
@Override
- public void registerIncomingBatchHandler(IncomingFragmentHandler handler) {
- this.handler.registerIncomingFragmentHandler(handler);
+ public ControlTunnel getTunnel(DrillbitEndpoint endpoint) {
+ return new ControlTunnel(endpoint, connectionRegistry.getConnectionManager(endpoint));
}
public void close() {
Closeables.closeQuietly(server);
- for (BitConnectionManager bt : connectionRegistry) {
+ for (ControlConnectionManager bt : connectionRegistry) {
bt.close();
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/DefaultInstanceHandler.java
index af3b10ac1..415932b2d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/DefaultInstanceHandler.java
@@ -15,20 +15,20 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc.control;
-import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
-import org.apache.drill.exec.proto.ExecProtos.BitStatus;
+import org.apache.drill.exec.proto.BitControl.BitControlHandshake;
+import org.apache.drill.exec.proto.BitControl.BitStatus;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.RpcType;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.rpc.RpcException;
import com.google.protobuf.MessageLite;
-public class BitComDefaultInstanceHandler {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComDefaultInstanceHandler.class);
+public class DefaultInstanceHandler {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DefaultInstanceHandler.class);
public static MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
@@ -36,7 +36,7 @@ public class BitComDefaultInstanceHandler {
case RpcType.ACK_VALUE:
return Ack.getDefaultInstance();
case RpcType.HANDSHAKE_VALUE:
- return BitHandshake.getDefaultInstance();
+ return BitControlHandshake.getDefaultInstance();
case RpcType.RESP_FRAGMENT_HANDLE_VALUE:
return FragmentHandle.getDefaultInstance();
case RpcType.RESP_FRAGMENT_STATUS_VALUE:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/SendProgress.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/SendProgress.java
new file mode 100644
index 000000000..f270eeedb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/SendProgress.java
@@ -0,0 +1,22 @@
+/**
+ * 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.rpc.control;
+
+public class SendProgress {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SendProgress.class);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/WorkEventBus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/WorkEventBus.java
new file mode 100644
index 000000000..5592e35fa
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/WorkEventBus.java
@@ -0,0 +1,113 @@
+/**
+ * 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.rpc.control;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.foreman.FragmentStatusListener;
+import org.apache.drill.exec.work.fragment.FragmentManager;
+import org.apache.drill.exec.work.fragment.NonRootFragmentManager;
+import org.apache.drill.exec.work.fragment.RootFragmentManager;
+
+import com.google.common.collect.Maps;
+
+public class WorkEventBus {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkEventBus.class);
+
+ private final ConcurrentMap<FragmentHandle, FragmentManager> managers = Maps.newConcurrentMap();
+ private final ConcurrentMap<QueryId, FragmentStatusListener> listeners = new ConcurrentHashMap<QueryId, FragmentStatusListener>(
+ 16, 0.75f, 16);
+ private final WorkerBee bee;
+
+ public WorkEventBus(WorkerBee bee) {
+ this.bee = bee;
+ }
+
+ public void removeFragmentStatusListener(FragmentHandle handle) throws RpcException {
+ logger.debug("Removing framgent status listener for handle {}.", handle);
+ listeners.remove(handle);
+ }
+
+ public void setFragmentStatusListener(QueryId queryId, FragmentStatusListener listener) throws RpcException {
+ logger.debug("Adding fragment status listener for queryId {}.", queryId);
+ FragmentStatusListener old = listeners.putIfAbsent(queryId, listener);
+ if (old != null)
+ throw new RpcException(
+ "Failure. The provided handle already exists in the listener pool. You need to remove one listener before adding another.");
+ }
+
+ public void status(FragmentStatus status) {
+ FragmentStatusListener l = listeners.get(status.getHandle().getQueryId());
+ if (l == null) {
+
+ logger.error("A fragment message arrived but there was no registered listener for that message for handle {}.",
+ status.getHandle());
+ return;
+ } else {
+ l.statusUpdate(status);
+ }
+ }
+
+ public void setRootFragmentManager(RootFragmentManager fragmentManager) {
+ FragmentManager old = managers.putIfAbsent(fragmentManager.getHandle(), fragmentManager);
+ if (old != null)
+ throw new IllegalStateException(
+ "Tried to set fragment manager when has already been set for the provided fragment handle.");
+ }
+
+ public FragmentManager getFragmentManager(FragmentHandle handle){
+ return managers.get(handle);
+ }
+
+ public FragmentManager getOrCreateFragmentManager(FragmentHandle handle) throws FragmentSetupException{
+ FragmentManager manager = managers.get(handle);
+ if (manager != null) return manager;
+
+ PlanFragment fragment = bee.getContext().getCache().getFragment(handle);
+
+ if (fragment == null) {
+ throw new FragmentSetupException("Received batch where fragment was not in cache.");
+ }
+
+ FragmentManager newManager = new NonRootFragmentManager(fragment, bee.getContext());
+
+ // since their could be a race condition on the check, we'll use putIfAbsent so we don't have two competing
+ // handlers.
+ manager = managers.putIfAbsent(fragment.getHandle(), newManager);
+
+ if (manager == null) {
+ // we added a handler, inform the bee that we did so. This way, the foreman can track status.
+ bee.addFragmentPendingRemote(newManager);
+ manager = newManager;
+ }
+
+ return manager;
+ }
+
+ public void removeFragmentManager(FragmentHandle handle){
+ managers.remove(handle);
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/BitServerConnection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/BitServerConnection.java
new file mode 100644
index 000000000..af4da41fe
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/BitServerConnection.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.exec.rpc.data;
+
+import io.netty.channel.Channel;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.work.fragment.FragmentManager;
+
+class BitServerConnection extends RemoteConnection{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitServerConnection.class);
+
+ private final BufferAllocator initialAllocator;
+ private volatile FragmentManager manager;
+
+ public BitServerConnection(Channel channel, BufferAllocator initialAllocator) {
+ super(channel);
+ this.initialAllocator = initialAllocator;
+ }
+
+ void setManager(FragmentManager manager){
+ this.manager = manager;
+ }
+
+ @Override
+ public BufferAllocator getAllocator() {
+ if(manager != null){
+ return manager.getFragmentContext().getAllocator();
+ }
+
+ return initialAllocator;
+
+ }
+
+ public FragmentManager getFragmentManager(){
+ return manager;
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java
new file mode 100644
index 000000000..6c57f22f7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java
@@ -0,0 +1,100 @@
+/**
+ * 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.rpc.data;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitData.BitClientHandshake;
+import org.apache.drill.exec.proto.BitData.BitServerHandshake;
+import org.apache.drill.exec.proto.BitData.RpcType;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.RpcChannel;
+import org.apache.drill.exec.rpc.BasicClient;
+import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.control.ControlProtobufLengthDecoder;
+import org.apache.drill.exec.server.BootStrapContext;
+
+import com.google.protobuf.MessageLite;
+
+public class DataClient extends BasicClient<RpcType, DataClientConnection, BitClientHandshake, BitServerHandshake>{
+
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataClient.class);
+
+ private final DrillbitEndpoint remoteEndpoint;
+ private volatile DataClientConnection connection;
+ private final BufferAllocator allocator;
+ private final DataConnectionManager.CloseHandlerCreator closeHandlerFactory;
+
+
+ public DataClient(DrillbitEndpoint remoteEndpoint, BootStrapContext context, DataConnectionManager.CloseHandlerCreator closeHandlerFactory) {
+ super(DataRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitClientLoopGroup(), RpcType.HANDSHAKE, BitServerHandshake.class, BitServerHandshake.PARSER);
+ this.remoteEndpoint = remoteEndpoint;
+ this.closeHandlerFactory = closeHandlerFactory;
+ this.allocator = context.getAllocator();
+ }
+
+ @Override
+ public DataClientConnection initRemoteConnection(Channel channel) {
+ this.connection = new DataClientConnection(channel, this);
+ return connection;
+ }
+
+ @Override
+ protected GenericFutureListener<ChannelFuture> getCloseHandler(DataClientConnection clientConnection) {
+ return closeHandlerFactory.getHandler(clientConnection, super.getCloseHandler(clientConnection));
+ }
+
+ @Override
+ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+ return DataDefaultInstanceHandler.getResponseDefaultInstanceClient(rpcType);
+ }
+
+ @Override
+ protected Response handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ throw new UnsupportedOperationException("DataClient is unidirectional by design.");
+ }
+
+ BufferAllocator getAllocator(){
+ return allocator;
+ }
+
+ @Override
+ protected void validateHandshake(BitServerHandshake handshake) throws RpcException {
+ if(handshake.getRpcVersion() != DataRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+ }
+
+ @Override
+ protected void finalizeConnection(BitServerHandshake handshake, DataClientConnection connection) {
+ }
+
+ public DataClientConnection getConnection(){
+ return this.connection;
+ }
+
+ @Override
+ public ProtobufLengthDecoder getDecoder(BufferAllocator allocator) {
+ return new DataProtobufLengthDecoder(allocator);
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClientConnection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClientConnection.java
new file mode 100644
index 000000000..9656a1409
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClientConnection.java
@@ -0,0 +1,80 @@
+/**
+ * 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.rpc.data;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+
+import java.util.UUID;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitData.RpcType;
+import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+
+import com.google.common.io.Closeables;
+import com.google.protobuf.MessageLite;
+
+public class DataClientConnection extends RemoteConnection{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataClientConnection.class);
+
+ private final DataClient client;
+ private final UUID id;
+
+ public DataClientConnection(Channel channel, DataClient client){
+ super(channel);
+ this.client = client;
+ // we use a local listener pool unless a global one is provided.
+ this.id = UUID.randomUUID();
+ }
+
+ @Override
+ public BufferAllocator getAllocator() {
+ return client.getAllocator();
+ }
+
+ public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> outcomeListener, RpcType rpcType,
+ SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies){
+ client.send(outcomeListener, this, rpcType, protobufBody, clazz, dataBodies);
+
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((id == null) ? 0 : id.hashCode());
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) return true;
+ if (obj == null) return false;
+ if (getClass() != obj.getClass()) return false;
+ DataClientConnection other = (DataClientConnection) obj;
+ if (id == null) {
+ if (other.id != null) return false;
+ } else if (!id.equals(other.id)) return false;
+ return true;
+ }
+
+ public void shutdownIfClient(){
+ Closeables.closeQuietly(client);
+ }
+} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
new file mode 100644
index 000000000..f15494fa6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
@@ -0,0 +1,63 @@
+/**
+ * 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.rpc.data;
+
+import java.io.Closeable;
+
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
+import org.apache.drill.exec.server.BootStrapContext;
+
+import com.google.common.io.Closeables;
+
+/**
+ * Manages a connection pool for each endpoint.
+ */
+public class DataConnectionCreator implements Closeable {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataConnectionCreator.class);
+
+ private volatile DataServer server;
+ private final BootStrapContext context;
+ private final WorkEventBus workBus;
+ private final DataResponseHandler dataHandler;
+
+ public DataConnectionCreator(BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler) {
+ super();
+ this.context = context;
+ this.workBus = workBus;
+ this.dataHandler = dataHandler;
+ }
+
+ public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws InterruptedException, DrillbitStartupException {
+ server = new DataServer(context, workBus, dataHandler);
+ int port = server.bind(partialEndpoint.getControlPort() + 1);
+ DrillbitEndpoint completeEndpoint = partialEndpoint.toBuilder().setDataPort(port).build();
+ return completeEndpoint;
+ }
+
+ public DataTunnel getTunnel(DrillbitEndpoint endpoint, FragmentHandle handle) {
+ return new DataTunnel(new DataConnectionManager(handle, endpoint, context));
+ }
+
+ public void close() {
+ Closeables.closeQuietly(server);
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionManager.java
new file mode 100644
index 000000000..bd0e9e044
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionManager.java
@@ -0,0 +1,53 @@
+/**
+ * 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.rpc.data;
+
+import org.apache.drill.exec.proto.BitData.BitClientHandshake;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.RpcChannel;
+import org.apache.drill.exec.rpc.ReconnectingConnection;
+import org.apache.drill.exec.server.BootStrapContext;
+
+public class DataConnectionManager extends ReconnectingConnection<DataClientConnection, BitClientHandshake>{
+
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataConnectionManager.class);
+
+ private final DrillbitEndpoint endpoint;
+ private final BootStrapContext context;
+
+ public DataConnectionManager(FragmentHandle handle, DrillbitEndpoint endpoint, BootStrapContext context) {
+ super(hs(handle), endpoint.getAddress(), endpoint.getDataPort());
+ this.endpoint = endpoint;
+ this.context = context;
+ }
+
+ @Override
+ protected DataClient getNewClient() {
+ return new DataClient(endpoint, context, new CloseHandlerCreator());
+ }
+
+ private static BitClientHandshake hs(FragmentHandle handle){
+ return BitClientHandshake //
+ .newBuilder() //
+ .setRpcVersion(DataRpcConfig.RPC_VERSION) //
+ .setChannel(RpcChannel.BIT_DATA) //
+ .setHandle(handle) //
+ .build();
+ }
+} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataDefaultInstanceHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataDefaultInstanceHandler.java
new file mode 100644
index 000000000..270dae63c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataDefaultInstanceHandler.java
@@ -0,0 +1,58 @@
+/**
+ * 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.rpc.data;
+
+import org.apache.drill.exec.proto.BitData.BitClientHandshake;
+import org.apache.drill.exec.proto.BitData.BitServerHandshake;
+import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
+import org.apache.drill.exec.proto.BitData.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.RpcException;
+
+import com.google.protobuf.MessageLite;
+
+public class DataDefaultInstanceHandler {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataDefaultInstanceHandler.class);
+
+
+ public static MessageLite getResponseDefaultInstanceClient(int rpcType) throws RpcException {
+ switch (rpcType) {
+ case RpcType.ACK_VALUE:
+ return Ack.getDefaultInstance();
+ case RpcType.HANDSHAKE_VALUE:
+ return BitServerHandshake.getDefaultInstance();
+
+ default:
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ public static MessageLite getResponseDefaultInstanceServer(int rpcType) throws RpcException {
+ switch (rpcType) {
+ case RpcType.ACK_VALUE:
+ return Ack.getDefaultInstance();
+ case RpcType.HANDSHAKE_VALUE:
+ return BitClientHandshake.getDefaultInstance();
+ case RpcType.REQ_RECORD_BATCH_VALUE:
+ return FragmentRecordBatch.getDefaultInstance();
+
+ default:
+ throw new UnsupportedOperationException();
+ }
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataProtobufLengthDecoder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataProtobufLengthDecoder.java
new file mode 100644
index 000000000..d4391480b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataProtobufLengthDecoder.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.rpc.data;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+
+import java.util.List;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
+
+public class DataProtobufLengthDecoder extends ProtobufLengthDecoder{
+
+ public DataProtobufLengthDecoder(BufferAllocator allocator) {
+ super(allocator);
+
+ }
+
+ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
+ super.decode(ctx, in, out);
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandler.java
new file mode 100644
index 000000000..aaff504d0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandler.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.rpc.data;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
+import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.work.fragment.FragmentManager;
+
+public interface DataResponseHandler {
+
+ public Response handle(RemoteConnection connection, FragmentManager manager, FragmentRecordBatch fragmentBatch, ByteBuf data) throws RpcException;
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandlerImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandlerImpl.java
new file mode 100644
index 000000000..b5b41a3ef
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandlerImpl.java
@@ -0,0 +1,64 @@
+/**
+ * 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.rpc.data;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
+import org.apache.drill.exec.proto.BitData.RpcType;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.fragment.FragmentManager;
+
+public class DataResponseHandlerImpl implements DataResponseHandler{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataResponseHandlerImpl.class);
+
+ private final WorkerBee bee;
+
+ public DataResponseHandlerImpl(WorkerBee bee) {
+ super();
+ this.bee = bee;
+ }
+
+ public Response handle(RemoteConnection connection, FragmentManager manager, FragmentRecordBatch fragmentBatch, ByteBuf data) throws RpcException {
+// logger.debug("Fragment Batch received {}", fragmentBatch);
+ try {
+ boolean canRun = manager.handle(new RawFragmentBatch(connection, fragmentBatch, data));
+ if (canRun) {
+// logger.debug("Arriving batch means local batch can run, starting local batch.");
+ // if we've reached the canRun threshold, we'll proceed. This expects handler.handle() to only return a single
+ // true.
+ bee.startFragmentPendingRemote(manager);
+ }
+ if (fragmentBatch.getIsLastBatch() && !manager.isWaiting()) {
+// logger.debug("Removing handler. Is Last Batch {}. Is Waiting for more {}", fragmentBatch.getIsLastBatch(),
+// manager.isWaiting());
+ bee.getContext().getWorkBus().removeFragmentManager(manager.getHandle());
+ }
+
+ return DataRpcConfig.OK;
+ } catch (FragmentSetupException e) {
+ logger.error("Failure while attempting to setup new fragment.", e);
+ return new Response(RpcType.ACK, Acks.FAIL);
+ }
+ }}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataRpcConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataRpcConfig.java
new file mode 100644
index 000000000..62248275f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataRpcConfig.java
@@ -0,0 +1,42 @@
+/**
+ * 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.rpc.data;
+
+
+import org.apache.drill.exec.proto.BitData.BitClientHandshake;
+import org.apache.drill.exec.proto.BitData.BitServerHandshake;
+import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
+import org.apache.drill.exec.proto.BitData.RpcType;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcConfig;
+
+public class DataRpcConfig {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataRpcConfig.class);
+
+ public static RpcConfig MAPPING = RpcConfig.newBuilder("BIT-DATA-RPC-MAPPING") //
+ .add(RpcType.HANDSHAKE, BitClientHandshake.class, RpcType.HANDSHAKE, BitServerHandshake.class)
+ .add(RpcType.REQ_RECORD_BATCH, FragmentRecordBatch.class, RpcType.ACK, Ack.class)
+ .build();
+
+ public static int RPC_VERSION = 1;
+
+ public static final Response OK = new Response(RpcType.ACK, Acks.OK);
+}
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
new file mode 100644
index 000000000..3dd7912cd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java
@@ -0,0 +1,128 @@
+/**
+ * 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.rpc.data;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitData.BitClientHandshake;
+import org.apache.drill.exec.proto.BitData.BitServerHandshake;
+import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
+import org.apache.drill.exec.proto.BitData.RpcType;
+import org.apache.drill.exec.proto.UserBitShared.RpcChannel;
+import org.apache.drill.exec.rpc.BasicServer;
+import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
+import org.apache.drill.exec.server.BootStrapContext;
+
+import com.google.protobuf.MessageLite;
+
+public class DataServer extends BasicServer<RpcType, BitServerConnection> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataServer.class);
+
+ private volatile ProxyCloseHandler proxyCloseHandler;
+ private final BootStrapContext context;
+ private final WorkEventBus workBus;
+ private final DataResponseHandler dataHandler;
+
+ public DataServer(BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler) {
+ super(DataRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup());
+ this.context = context;
+ this.workBus = workBus;
+ this.dataHandler = dataHandler;
+ }
+
+ @Override
+ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+ return DataDefaultInstanceHandler.getResponseDefaultInstanceServer(rpcType);
+ }
+
+ @Override
+ protected GenericFutureListener<ChannelFuture> getCloseHandler(BitServerConnection connection) {
+ this.proxyCloseHandler = new ProxyCloseHandler(super.getCloseHandler(connection));
+ return proxyCloseHandler;
+ }
+
+ @Override
+ public BitServerConnection initRemoteConnection(Channel channel) {
+ return new BitServerConnection(channel, context.getAllocator());
+ }
+
+ @Override
+ protected ServerHandshakeHandler<BitClientHandshake> getHandshakeHandler(final BitServerConnection connection) {
+ return new ServerHandshakeHandler<BitClientHandshake>(RpcType.HANDSHAKE, BitClientHandshake.PARSER) {
+
+ @Override
+ public MessageLite getHandshakeResponse(BitClientHandshake inbound) throws Exception {
+ // logger.debug("Handling handshake from other bit. {}", inbound);
+ if (inbound.getRpcVersion() != DataRpcConfig.RPC_VERSION)
+ throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.",
+ inbound.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+ if (inbound.getChannel() != RpcChannel.BIT_DATA)
+ throw new RpcException(String.format("Invalid NodeMode. Expected BIT_DATA but received %s.",
+ inbound.getChannel()));
+
+ connection.setManager(workBus.getOrCreateFragmentManager(inbound.getHandle()));
+ return BitServerHandshake.newBuilder().setRpcVersion(DataRpcConfig.RPC_VERSION).build();
+ }
+
+ };
+ }
+
+ @Override
+ protected Response handle(BitServerConnection connection, int rpcType, ByteBuf pBody, ByteBuf body) throws RpcException {
+ assert rpcType == RpcType.REQ_RECORD_BATCH_VALUE;
+
+ FragmentRecordBatch fragmentBatch = get(pBody, FragmentRecordBatch.PARSER);
+ return dataHandler.handle(connection, connection.getFragmentManager(), fragmentBatch, body);
+ }
+
+ private class ProxyCloseHandler implements GenericFutureListener<ChannelFuture> {
+
+ private volatile GenericFutureListener<ChannelFuture> handler;
+
+ public ProxyCloseHandler(GenericFutureListener<ChannelFuture> handler) {
+ super();
+ this.handler = handler;
+ }
+
+ public GenericFutureListener<ChannelFuture> getHandler() {
+ return handler;
+ }
+
+ public void setHandler(GenericFutureListener<ChannelFuture> handler) {
+ this.handler = handler;
+ }
+
+ @Override
+ public void operationComplete(ChannelFuture future) throws Exception {
+ handler.operationComplete(future);
+ }
+
+ }
+
+ @Override
+ public ProtobufLengthDecoder getDecoder(BufferAllocator allocator) {
+ return new DataProtobufLengthDecoder(allocator);
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java
new file mode 100644
index 000000000..1dcd89e9a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.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.rpc.data;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.BitData.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.record.FragmentWritableBatch;
+import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.rpc.FutureBitCommand;
+import org.apache.drill.exec.rpc.ListeningCommand;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+
+
+public class DataTunnel {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataTunnel.class);
+
+ private final DataConnectionManager manager;
+
+ public DataTunnel(DataConnectionManager manager) {
+ this.manager = manager;
+ }
+
+ public void sendRecordBatch(RpcOutcomeListener<Ack> outcomeListener, FragmentWritableBatch batch) {
+ SendBatch b = new SendBatch(outcomeListener, batch);
+ manager.runCommand(b);
+ }
+
+ public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, FragmentWritableBatch batch) {
+ SendBatchAsync b = new SendBatchAsync(batch, context);
+ manager.runCommand(b);
+ return b.getFuture();
+ }
+
+
+
+ public static class SendBatch extends ListeningCommand<Ack, DataClientConnection> {
+ final FragmentWritableBatch batch;
+
+ public SendBatch(RpcOutcomeListener<Ack> listener, FragmentWritableBatch batch) {
+ super(listener);
+ this.batch = batch;
+ }
+
+ @Override
+ public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, DataClientConnection connection) {
+ connection.send(outcomeListener, RpcType.REQ_RECORD_BATCH, batch.getHeader(), Ack.class, batch.getBuffers());
+ }
+
+ @Override
+ public String toString() {
+ return "SendBatch [batch.header=" + batch.getHeader() + "]";
+ }
+
+
+ }
+
+ public static class SendBatchAsync extends FutureBitCommand<Ack, DataClientConnection> {
+ final FragmentWritableBatch batch;
+ final FragmentContext context;
+
+ public SendBatchAsync(FragmentWritableBatch batch, FragmentContext context) {
+ super();
+ this.batch = batch;
+ this.context = context;
+ }
+
+ @Override
+ public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, DataClientConnection connection) {
+ connection.send(outcomeListener, RpcType.REQ_RECORD_BATCH, batch.getHeader(), Ack.class, batch.getBuffers());
+ }
+
+ @Override
+ public String toString() {
+ return "SendBatch [batch.header=" + batch.getHeader() + "]";
+ }
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/SendProgress.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/SendProgress.java
index b26adb198..490013fa8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/SendProgress.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/SendProgress.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.rpc.bit;
+package org.apache.drill.exec.rpc.data;
public class SendProgress {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SendProgress.class);
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 392a168b3..f28ff4b81 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
@@ -18,9 +18,9 @@
package org.apache.drill.exec.rpc.user;
import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
import io.netty.channel.EventLoopGroup;
+import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
@@ -30,6 +30,7 @@ import org.apache.drill.exec.proto.UserProtos.RpcType;
import org.apache.drill.exec.proto.UserProtos.RunQuery;
import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
import org.apache.drill.exec.rpc.BasicClientWithConnection;
+import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.RpcConnectionHandler;
import org.apache.drill.exec.rpc.RpcException;
@@ -41,16 +42,12 @@ public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHand
private final QueryResultHandler queryResultHandler = new QueryResultHandler();
- public UserClient(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+ public UserClient(BufferAllocator alloc, EventLoopGroup eventLoopGroup) {
super(UserRpcConfig.MAPPING, alloc, eventLoopGroup, RpcType.HANDSHAKE, BitToUserHandshake.class, BitToUserHandshake.PARSER);
}
public void submitQuery(UserResultsListener resultsListener, RunQuery query) {
- try{
- send(queryResultHandler.getWrappedListener(resultsListener), RpcType.RUN_QUERY, query, QueryId.class);
- }catch(RpcException ex){
- resultsListener.submissionFailed(ex);
- }
+ send(queryResultHandler.getWrappedListener(resultsListener), RpcType.RUN_QUERY, query, QueryId.class);
}
public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint) throws RpcException, InterruptedException {
@@ -86,7 +83,7 @@ public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHand
@Override
protected void validateHandshake(BitToUserHandshake inbound) throws RpcException {
- logger.debug("Handling handshake from bit to user. {}", inbound);
+// logger.debug("Handling handshake from bit to user. {}", inbound);
if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION)
throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", inbound.getRpcVersion(),
UserRpcConfig.RPC_VERSION));
@@ -96,5 +93,10 @@ public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHand
@Override
protected void finalizeConnection(BitToUserHandshake handshake, BasicClientWithConnection.ServerConnection connection) {
}
+
+ @Override
+ public ProtobufLengthDecoder getDecoder(BufferAllocator allocator) {
+ return new UserProtobufLengthDecoder(allocator);
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserProtobufLengthDecoder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserProtobufLengthDecoder.java
new file mode 100644
index 000000000..680a07d49
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserProtobufLengthDecoder.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.rpc.user;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+
+import java.util.List;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
+
+public class UserProtobufLengthDecoder extends ProtobufLengthDecoder{
+
+ public UserProtobufLengthDecoder(BufferAllocator allocator) {
+ super(allocator);
+
+ }
+ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
+ super.decode(ctx, in, out);
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index b973be933..0f45dbee7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -23,6 +23,7 @@ import io.netty.buffer.ByteBufInputStream;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
+import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
@@ -31,10 +32,12 @@ import org.apache.drill.exec.proto.UserProtos.RpcType;
import org.apache.drill.exec.proto.UserProtos.RunQuery;
import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
import org.apache.drill.exec.rpc.BasicServer;
+import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
import org.apache.drill.exec.rpc.RemoteConnection;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.rpc.data.DataProtobufLengthDecoder;
import org.apache.drill.exec.work.user.UserWorker;
import com.google.protobuf.InvalidProtocolBufferException;
@@ -44,10 +47,12 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
final UserWorker worker;
+ final BufferAllocator alloc;
- public UserServer(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, UserWorker worker) {
- super(UserRpcConfig.MAPPING, alloc, eventLoopGroup);
+ public UserServer(BufferAllocator alloc, EventLoopGroup eventLoopGroup, UserWorker worker) {
+ super(UserRpcConfig.MAPPING, alloc.getUnderlyingAllocator(), eventLoopGroup);
this.worker = worker;
+ this.alloc = alloc;
}
@Override
@@ -94,16 +99,23 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
}
+
public class UserClientConnection extends RemoteConnection {
public UserClientConnection(Channel channel) {
super(channel);
}
public void sendResult(RpcOutcomeListener<Ack> listener, QueryWritableBatch result){
- logger.debug("Sending result to client with {}", result);
+// logger.debug("Sending result to client with {}", result);
+
send(listener, this, RpcType.QUERY_RESULT, result.getHeader(), Ack.class, result.getBuffers());
}
+ @Override
+ public BufferAllocator getAllocator() {
+ return alloc;
+ }
+
}
@Override
@@ -117,7 +129,7 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
@Override
public MessageLite getHandshakeResponse(UserToBitHandshake inbound) throws Exception {
- logger.debug("Handling handshake from user to bit. {}", inbound);
+// logger.debug("Handling handshake from user to bit. {}", inbound);
if(inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", inbound.getRpcVersion(), UserRpcConfig.RPC_VERSION));
return BitToUserHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).build();
}
@@ -125,5 +137,8 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
};
}
-
+ @Override
+ public ProtobufLengthDecoder getDecoder(BufferAllocator allocator) {
+ return new UserProtobufLengthDecoder(allocator);
+ }
}
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 225bd0b1c..016d328a4 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
@@ -24,6 +24,7 @@ import java.io.Closeable;
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.rpc.NamedThreadFactory;
import com.codahale.metrics.MetricRegistry;
@@ -33,6 +34,7 @@ public class BootStrapContext implements Closeable{
private final DrillConfig config;
private final NioEventLoopGroup loop;
+ private final NioEventLoopGroup loop2;
private final MetricRegistry metrics;
private final BufferAllocator allocator;
@@ -40,8 +42,9 @@ public class BootStrapContext implements Closeable{
super();
this.config = config;
this.loop = new NioEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), new NamedThreadFactory("BitServer-"));
+ this.loop2 = new NioEventLoopGroup(config.getInt(ExecConstants.BIT_SERVER_RPC_THREADS), new NamedThreadFactory("BitClient-"));
this.metrics = new MetricRegistry();
- this.allocator = BufferAllocator.getAllocator(config);
+ this.allocator = new TopLevelAllocator();
}
public DrillConfig getConfig() {
@@ -52,6 +55,10 @@ public class BootStrapContext implements Closeable{
return loop;
}
+ public NioEventLoopGroup getBitClientLoopGroup() {
+ return loop2;
+ }
+
public MetricRegistry getMetrics() {
return metrics;
}
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 49732d79c..6400c7575 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
@@ -78,14 +78,14 @@ public class Drillbit implements Closeable{
this.context = new BootStrapContext(config);
this.manager = new WorkManager(context);
this.coord = serviceSet.getCoordinator();
- this.engine = new ServiceEngine(manager.getBitComWorker(), manager.getUserWorker(), context);
+ this.engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context, manager.getWorkBus(), manager.getDataHandler());
this.cache = serviceSet.getCache();
}else{
Runtime.getRuntime().addShutdownHook(new ShutdownThread(config));
this.context = new BootStrapContext(config);
this.manager = new WorkManager(context);
this.coord = new ZKClusterCoordinator(config);
- this.engine = new ServiceEngine(manager.getBitComWorker(), manager.getUserWorker(), context);
+ this.engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context, manager.getWorkBus(), manager.getDataHandler());
this.cache = new HazelCache(config, context.getAllocator());
}
}
@@ -93,7 +93,7 @@ public class Drillbit implements Closeable{
public void run() throws Exception {
coord.start(10000);
DrillbitEndpoint md = engine.start();
- manager.start(md, cache, engine.getBitCom(), coord);
+ manager.start(md, cache, engine.getController(), engine.getDataConnectionCreator(), coord);
cache.run();
handle = coord.register(md);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index 465659042..39821e39f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -30,7 +30,9 @@ import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.control.Controller;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
+import org.apache.drill.exec.rpc.data.DataConnectionCreator;
import org.apache.drill.exec.store.StorageEngine;
import org.apache.drill.exec.store.StorageEngineRegistry;
@@ -44,21 +46,25 @@ public class DrillbitContext {
private PhysicalPlanReader reader;
private final ClusterCoordinator coord;
- private final BitCom com;
+ private final DataConnectionCreator connectionsPool;
private final DistributedCache cache;
private final DrillbitEndpoint endpoint;
private final StorageEngineRegistry storageEngineRegistry;
private final OperatorCreatorRegistry operatorCreatorRegistry;
+ private final Controller controller;
+ private final WorkEventBus workBus;
- public DrillbitContext(DrillbitEndpoint endpoint, BootStrapContext context, ClusterCoordinator coord, BitCom com, DistributedCache cache) {
+ public DrillbitContext(DrillbitEndpoint endpoint, BootStrapContext context, ClusterCoordinator coord, Controller controller, DataConnectionCreator connectionsPool, DistributedCache cache, WorkEventBus workBus) {
super();
Preconditions.checkNotNull(endpoint);
Preconditions.checkNotNull(context);
- Preconditions.checkNotNull(com);
-
+ Preconditions.checkNotNull(controller);
+ Preconditions.checkNotNull(connectionsPool);
+ this.workBus = workBus;
+ this.controller = controller;
this.context = context;
this.coord = coord;
- this.com = com;
+ this.connectionsPool = connectionsPool;
this.cache = cache;
this.endpoint = endpoint;
this.storageEngineRegistry = new StorageEngineRegistry(this);
@@ -66,6 +72,10 @@ public class DrillbitContext {
this.operatorCreatorRegistry = new OperatorCreatorRegistry(context.getConfig());
}
+ public WorkEventBus getWorkBus(){
+ return workBus;
+ }
+
public DrillbitEndpoint getEndpoint(){
return endpoint;
}
@@ -94,8 +104,13 @@ public class DrillbitContext {
return context.getBitLoopGroup();
}
- public BitCom getBitCom(){
- return com;
+
+ public DataConnectionCreator getDataConnectionsPool(){
+ return connectionsPool;
+ }
+
+ public Controller getController(){
+ return controller;
}
public MetricRegistry getMetrics(){
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
index 8406debc0..cfbde7395 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
@@ -29,11 +29,14 @@ import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.exception.DrillbitStartupException;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.rpc.NamedThreadFactory;
-import org.apache.drill.exec.rpc.bit.BitCom;
-import org.apache.drill.exec.rpc.bit.BitComImpl;
+import org.apache.drill.exec.rpc.control.Controller;
+import org.apache.drill.exec.rpc.control.ControllerImpl;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
+import org.apache.drill.exec.rpc.data.DataConnectionCreator;
+import org.apache.drill.exec.rpc.data.DataResponseHandler;
import org.apache.drill.exec.rpc.user.UserServer;
import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
import org.apache.drill.exec.work.user.UserWorker;
import com.google.common.io.Closeables;
@@ -42,14 +45,16 @@ public class ServiceEngine implements Closeable{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ServiceEngine.class);
private final UserServer userServer;
- private final BitCom bitCom;
+ private final Controller controller;
+ private final DataConnectionCreator dataPool;
private final DrillConfig config;
boolean useIP = false;
- public ServiceEngine(BitComHandler bitComWorker, UserWorker userWorker, BootStrapContext context){
- this.userServer = new UserServer(context.getAllocator().getUnderlyingAllocator(), new NioEventLoopGroup(context.getConfig().getInt(ExecConstants.USER_SERVER_RPC_THREADS),
+ public ServiceEngine(ControlMessageHandler controlMessageHandler, UserWorker userWorker, BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler){
+ this.userServer = new UserServer(context.getAllocator(), new NioEventLoopGroup(context.getConfig().getInt(ExecConstants.USER_SERVER_RPC_THREADS),
new NamedThreadFactory("UserServer-")), userWorker);
- this.bitCom = new BitComImpl(context, bitComWorker);
+ this.controller = new ControllerImpl(context, controlMessageHandler);
+ this.dataPool = new DataConnectionCreator(context, workBus, dataHandler);
this.config = context.getConfig();
}
@@ -62,16 +67,22 @@ public class ServiceEngine implements Closeable{
.setUserPort(userPort)
.build();
- return bitCom.start(partialEndpoint);
+ partialEndpoint = controller.start(partialEndpoint);
+ return dataPool.start(partialEndpoint);
}
- public BitCom getBitCom(){
- return bitCom;
+ public DataConnectionCreator getDataConnectionCreator(){
+ return dataPool;
}
+ public Controller getController() {
+ return controller;
+ }
+
@Override
public void close() throws IOException {
Closeables.closeQuietly(userServer);
- Closeables.closeQuietly(bitCom);
+ Closeables.closeQuietly(dataPool);
+ Closeables.closeQuietly(controller);
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
index 491779ae0..9a6b7fff9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
@@ -384,7 +384,7 @@ public class ParquetRecordReader implements RecordReader {
// cannot find more information on this right now, will keep looking
}
- logger.debug("records to read in this pass: {}", recordsToRead);
+// logger.debug("records to read in this pass: {}", recordsToRead);
if (allFieldsFixedLength) {
readAllFixedFields(recordsToRead, firstColumnStatus);
} else { // variable length columns
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/AssertionUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/AssertionUtil.java
new file mode 100644
index 000000000..a10038a01
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/AssertionUtil.java
@@ -0,0 +1,35 @@
+/**
+ * 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.util;
+
+public class AssertionUtil {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AssertionUtil.class);
+
+
+ private static final boolean ASSERT_ENABLED;
+
+ static{
+ boolean isAssertEnabled = false;
+ assert isAssertEnabled = true;
+ ASSERT_ENABLED = isAssertEnabled;
+ }
+
+ public static boolean isAssertionsEnabled(){
+ return ASSERT_ENABLED;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
index f6653bc12..2df199d8b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
@@ -19,8 +19,6 @@ package org.apache.drill.exec.util;
import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
-
import com.google.protobuf.Internal.EnumLite;
/**
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 aa2826feb..cbc8b86c7 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
@@ -68,7 +68,6 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
valueCapacity = valueCount;
int valueSize = getSizeFromCount(valueCount);
data = allocator.buffer(valueSize);
- this.data.retain();
for (int i = 0; i < valueSize; i++) {
data.setByte(i, 0);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/ErrorHelper.java
index 26ecc3b7d..8a106127c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/ErrorHelper.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.work.foreman;
+package org.apache.drill.exec.work;
import java.util.UUID;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
index f001012e1..c3760d59a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
@@ -20,8 +20,7 @@ package org.apache.drill.exec.work;
import java.util.List;
import org.apache.drill.exec.physical.base.FragmentRoot;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import com.google.common.base.Preconditions;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java
index 3ddd2ac8f..9215f43d7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java
@@ -17,7 +17,7 @@
*/
package org.apache.drill.exec.work;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
public interface StatusProvider {
public FragmentStatus getStatus();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index 9a82c6298..29f011f62 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -35,13 +35,18 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.rpc.NamedThreadFactory;
-import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.control.Controller;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
+import org.apache.drill.exec.rpc.data.DataConnectionCreator;
+import org.apache.drill.exec.rpc.data.DataResponseHandler;
+import org.apache.drill.exec.rpc.data.DataResponseHandlerImpl;
import org.apache.drill.exec.server.BootStrapContext;
import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.work.batch.BitComHandler;
-import org.apache.drill.exec.work.batch.BitComHandlerImpl;
+import org.apache.drill.exec.work.batch.ControlHandlerImpl;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
import org.apache.drill.exec.work.foreman.Foreman;
-import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.apache.drill.exec.work.fragment.FragmentExecutor;
+import org.apache.drill.exec.work.fragment.FragmentManager;
import org.apache.drill.exec.work.user.UserWorker;
import com.google.common.collect.Maps;
@@ -50,39 +55,51 @@ import com.google.common.collect.Queues;
public class WorkManager implements Closeable{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkManager.class);
- private Set<IncomingFragmentHandler> incomingFragments = Collections.newSetFromMap(Maps.<IncomingFragmentHandler, Boolean> newConcurrentMap());
+ private Set<FragmentManager> incomingFragments = Collections.newSetFromMap(Maps.<FragmentManager, Boolean> newConcurrentMap());
private PriorityBlockingQueue<Runnable> pendingTasks = Queues.newPriorityBlockingQueue();
- private Map<FragmentHandle, FragmentRunner> runningFragments = Maps.newConcurrentMap();
+ private Map<FragmentHandle, FragmentExecutor> runningFragments = Maps.newConcurrentMap();
private ConcurrentMap<QueryId, Foreman> queries = Maps.newConcurrentMap();
private BootStrapContext bContext;
private DrillbitContext dContext;
- private final BitComHandler bitComWorker;
+ private final ControlMessageHandler controlMessageWorker;
+ private final DataResponseHandler dataHandler;
private final UserWorker userWorker;
private final WorkerBee bee;
+ private final WorkEventBus workBus;
private Executor executor;
private final EventThread eventThread;
public WorkManager(BootStrapContext context){
this.bee = new WorkerBee();
+ this.workBus = new WorkEventBus(bee);
this.bContext = context;
- this.bitComWorker = new BitComHandlerImpl(bee);
+ this.controlMessageWorker = new ControlHandlerImpl(bee);
this.userWorker = new UserWorker(bee);
this.eventThread = new EventThread();
+ this.dataHandler = new DataResponseHandlerImpl(bee);
}
- public void start(DrillbitEndpoint endpoint, DistributedCache cache, BitCom com, ClusterCoordinator coord){
- this.dContext = new DrillbitContext(endpoint, bContext, coord, com, cache);
+ public void start(DrillbitEndpoint endpoint, DistributedCache cache, Controller controller, DataConnectionCreator data, ClusterCoordinator coord){
+ this.dContext = new DrillbitContext(endpoint, bContext, coord, controller, data, cache, workBus);
executor = Executors.newFixedThreadPool(dContext.getConfig().getInt(ExecConstants.EXECUTOR_THREADS), new NamedThreadFactory("WorkManager-"));
eventThread.start();
}
- public BitComHandler getBitComWorker(){
- return bitComWorker;
+ public WorkEventBus getWorkBus(){
+ return workBus;
+ }
+
+ public DataResponseHandler getDataHandler() {
+ return dataHandler;
+ }
+
+ public ControlMessageHandler getControlMessageHandler(){
+ return controlMessageWorker;
}
public UserWorker getUserWorker(){
@@ -101,7 +118,7 @@ public class WorkManager implements Closeable{
// create this so items can see the data here whether or not they are in this package.
public class WorkerBee{
- public void addFragmentRunner(FragmentRunner runner){
+ public void addFragmentRunner(FragmentExecutor runner){
pendingTasks.add(runner);
}
@@ -110,16 +127,16 @@ public class WorkManager implements Closeable{
}
- public void addFragmentPendingRemote(IncomingFragmentHandler handler){
+ public void addFragmentPendingRemote(FragmentManager handler){
incomingFragments.add(handler);
}
- public void startFragmentPendingRemote(IncomingFragmentHandler handler){
+ public void startFragmentPendingRemote(FragmentManager handler){
incomingFragments.remove(handler);
pendingTasks.add(handler.getRunnable());
}
- public FragmentRunner getFragmentRunner(FragmentHandle handle){
+ public FragmentExecutor getFragmentRunner(FragmentHandle handle){
return runningFragments.get(handle);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java
index 702337349..30e6df285 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java
@@ -24,18 +24,16 @@ import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicIntegerArray;
-import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.exception.SetupException;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.base.Receiver;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.rpc.RemoteConnection;
import com.google.common.base.Preconditions;
-public abstract class AbstractFragmentCollector implements BatchCollector{
+public abstract class AbstractDataCollector implements DataCollector{
private final List<DrillbitEndpoint> incoming;
private final int oppositeMajorFragmentId;
@@ -45,7 +43,7 @@ public abstract class AbstractFragmentCollector implements BatchCollector{
private final AtomicInteger parentAccounter;
private final AtomicInteger finishedStreams = new AtomicInteger();
- public AbstractFragmentCollector(AtomicInteger parentAccounter, Receiver receiver, int minInputsRequired, FragmentContext context) {
+ public AbstractDataCollector(AtomicInteger parentAccounter, Receiver receiver, int minInputsRequired, FragmentContext context) {
Preconditions.checkArgument(minInputsRequired > 0);
Preconditions.checkNotNull(receiver);
Preconditions.checkNotNull(parentAccounter);
@@ -56,7 +54,7 @@ public abstract class AbstractFragmentCollector implements BatchCollector{
this.oppositeMajorFragmentId = receiver.getOppositeMajorFragmentId();
this.buffers = new RawBatchBuffer[minInputsRequired];
try {
- String bufferClassName = context.getConfig().getString(ExecConstants.SPOOLING_BUFFER_IMPL);
+ String bufferClassName = context.getConfig().getString(ExecConstants.INCOMING_BUFFER_IMPL);
Constructor<?> bufferConstructor = Class.forName(bufferClassName).getConstructor(FragmentContext.class);
for(int i = 0; i < buffers.length; i++) {
buffers[i] = (RawBatchBuffer) bufferConstructor.newInstance(context);
@@ -82,7 +80,7 @@ public abstract class AbstractFragmentCollector implements BatchCollector{
public abstract void streamFinished(int minorFragmentId);
- public boolean batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch) throws IOException {
+ public boolean batchArrived(int minorFragmentId, RawFragmentBatch batch) throws IOException {
boolean decremented = false;
if (remainders.compareAndSet(minorFragmentId, 0, 1)) {
int rem = remainingRequired.decrementAndGet();
@@ -94,7 +92,7 @@ public abstract class AbstractFragmentCollector implements BatchCollector{
if(batch.getHeader().getIsLastBatch()){
streamFinished(minorFragmentId);
}
- getBuffer(minorFragmentId).enqueue(throttle, batch);
+ getBuffer(minorFragmentId).enqueue(batch);
return decremented;
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
deleted file mode 100644
index 5d20026d3..000000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
+++ /dev/null
@@ -1,211 +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.work.batch;
-
-import static org.apache.drill.exec.rpc.RpcBus.get;
-import io.netty.buffer.ByteBuf;
-
-import java.io.IOException;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.base.FragmentRoot;
-import org.apache.drill.exec.physical.impl.ImplCreator;
-import org.apache.drill.exec.physical.impl.RootExec;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.rpc.Acks;
-import org.apache.drill.exec.rpc.RemoteConnection;
-import org.apache.drill.exec.rpc.Response;
-import org.apache.drill.exec.rpc.RpcConstants;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitConnection;
-import org.apache.drill.exec.rpc.bit.BitRpcConfig;
-import org.apache.drill.exec.rpc.bit.BitTunnel;
-import org.apache.drill.exec.work.FragmentRunner;
-import org.apache.drill.exec.work.RemoteFragmentRunnerListener;
-import org.apache.drill.exec.work.WorkManager.WorkerBee;
-import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
-import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
-
-import com.google.common.collect.Maps;
-
-public class BitComHandlerImpl implements BitComHandler {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComHandlerImpl.class);
-
- private ConcurrentMap<FragmentHandle, IncomingFragmentHandler> handlers = Maps.newConcurrentMap();
- private final WorkerBee bee;
-
- public BitComHandlerImpl(WorkerBee bee) {
- super();
- this.bee = bee;
- }
-
- /* (non-Javadoc)
- * @see org.apache.drill.exec.work.batch.BitComHandler#handle(org.apache.drill.exec.rpc.bit.BitConnection, int, io.netty.buffer.ByteBuf, io.netty.buffer.ByteBuf)
- */
- @Override
- public Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Received bit com message of type {}", rpcType);
-
- switch (rpcType) {
-
- case RpcType.REQ_CANCEL_FRAGMENT_VALUE:
- FragmentHandle handle = get(pBody, FragmentHandle.PARSER);
- cancelFragment(handle);
- return BitRpcConfig.OK;
-
- case RpcType.REQ_FRAGMENT_STATUS_VALUE:
- connection.getListenerPool().status( get(pBody, FragmentStatus.PARSER));
- // TODO: Support a type of message that has no response.
- return BitRpcConfig.OK;
-
- case RpcType.REQ_INIATILIZE_FRAGMENT_VALUE:
- PlanFragment fragment = get(pBody, PlanFragment.PARSER);
- startNewRemoteFragment(fragment);
- return BitRpcConfig.OK;
-
- case RpcType.REQ_RECORD_BATCH_VALUE:
- try {
- FragmentRecordBatch header = get(pBody, FragmentRecordBatch.PARSER);
- incomingRecordBatch(connection, header, dBody);
- return BitRpcConfig.OK;
- } catch (FragmentSetupException e) {
- throw new RpcException("Failure receiving record batch.", e);
- }
-
- default:
- throw new RpcException("Not yet supported.");
- }
-
- }
-
-
-
- /* (non-Javadoc)
- * @see org.apache.drill.exec.work.batch.BitComHandler#startNewRemoteFragment(org.apache.drill.exec.proto.ExecProtos.PlanFragment)
- */
- @Override
- public void startNewRemoteFragment(PlanFragment fragment){
- logger.debug("Received remote fragment start instruction", fragment);
- FragmentContext context = new FragmentContext(bee.getContext(), fragment.getHandle(), null, new FunctionImplementationRegistry(bee.getContext().getConfig()));
- BitTunnel tunnel = bee.getContext().getBitCom().getTunnel(fragment.getForeman());
- RemoteFragmentRunnerListener listener = new RemoteFragmentRunnerListener(context, tunnel);
- try{
- FragmentRoot rootOperator = bee.getContext().getPlanReader().readFragmentOperator(fragment.getFragmentJson());
- RootExec exec = ImplCreator.getExec(context, rootOperator);
- FragmentRunner fr = new FragmentRunner(context, exec, listener);
- bee.addFragmentRunner(fr);
-
- }catch(IOException e){
- listener.fail(fragment.getHandle(), "Failure while parsing fragment execution plan.", e);
- }catch(ExecutionSetupException e){
- listener.fail(fragment.getHandle(), "Failure while setting up execution plan.", e);
- } catch (Exception e) {
- listener.fail(fragment.getHandle(), "Failure due to uncaught exception", e);
- } catch (OutOfMemoryError t) {
- if(t.getMessage().startsWith("Direct buffer")){
- listener.fail(fragment.getHandle(), "Failure due to error", t);
- }else{
- throw t;
- }
-
- }
-
- }
-
- /* (non-Javadoc)
- * @see org.apache.drill.exec.work.batch.BitComHandler#cancelFragment(org.apache.drill.exec.proto.ExecProtos.FragmentHandle)
- */
- @Override
- public Ack cancelFragment(FragmentHandle handle){
- IncomingFragmentHandler handler = handlers.get(handle);
- if(handler != null){
- // try remote fragment cancel.
- handler.cancel();
- }else{
- // then try local cancel.
- FragmentRunner runner = bee.getFragmentRunner(handle);
- if(runner != null) runner.cancel();
- }
-
- return Acks.OK;
- }
-
-
- /**
- * Returns a positive Ack if this fragment is accepted.
- */
- private Ack incomingRecordBatch(RemoteConnection connection, FragmentRecordBatch fragmentBatch, ByteBuf body) throws FragmentSetupException{
- FragmentHandle handle = fragmentBatch.getHandle();
- IncomingFragmentHandler handler = handlers.get(handle);
-
- // Create a handler if there isn't already one.
- if(handler == null){
-
- PlanFragment fragment = bee.getContext().getCache().getFragment(handle);
- if(fragment == null){
- logger.error("Received batch where fragment was not in cache.");
- return Acks.FAIL;
- }
-
- IncomingFragmentHandler newHandler = new RemoteFragmentHandler(fragment, bee.getContext(), bee.getContext().getBitCom().getTunnel(fragment.getForeman()));
-
- // since their could be a race condition on the check, we'll use putIfAbsent so we don't have two competing handlers.
- handler = handlers.putIfAbsent(fragment.getHandle(), newHandler);
-
- if(handler == null){
- // we added a handler, inform the bee that we did so. This way, the foreman can track status.
- bee.addFragmentPendingRemote(newHandler);
- handler = newHandler;
- }
- }
-
- boolean canRun = handler.handle(connection.getConnectionThrottle(), new RawFragmentBatch(fragmentBatch, body));
- if(canRun){
- logger.debug("Arriving batch means local batch can run, starting local batch.");
- // if we've reached the canRun threshold, we'll proceed. This expects handler.handle() to only return a single true.
- bee.startFragmentPendingRemote(handler);
- }
- if(fragmentBatch.getIsLastBatch() && !handler.isWaiting()){
- logger.debug("Removing handler. Is Last Batch {}. Is Waiting for more {}", fragmentBatch.getIsLastBatch(), handler.isWaiting());
- handlers.remove(handler.getHandle());
- }
-
- return Acks.OK;
- }
-
- /* (non-Javadoc)
- * @see org.apache.drill.exec.work.batch.BitComHandler#registerIncomingFragmentHandler(org.apache.drill.exec.work.fragment.IncomingFragmentHandler)
- */
- @Override
- public void registerIncomingFragmentHandler(IncomingFragmentHandler handler){
- IncomingFragmentHandler old = handlers.putIfAbsent(handler.getHandle(), handler);
- assert old == null : "You can only register a fragment handler if one hasn't been registered already.";
- }
-
-
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlHandlerImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlHandlerImpl.java
new file mode 100644
index 000000000..92614ca22
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlHandlerImpl.java
@@ -0,0 +1,150 @@
+/**
+ * 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.work.batch;
+
+import static org.apache.drill.exec.rpc.RpcBus.get;
+import io.netty.buffer.ByteBuf;
+
+import java.io.IOException;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.memory.OutOfMemoryException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.RpcType;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcConstants;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.control.ControlConnection;
+import org.apache.drill.exec.rpc.control.ControlTunnel;
+import org.apache.drill.exec.rpc.data.DataRpcConfig;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.fragment.FragmentExecutor;
+import org.apache.drill.exec.work.fragment.FragmentManager;
+import org.apache.drill.exec.work.fragment.NonRootStatusReporter;
+
+public class ControlHandlerImpl implements ControlMessageHandler {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlHandlerImpl.class);
+
+ private final WorkerBee bee;
+
+ public ControlHandlerImpl(WorkerBee bee) {
+ super();
+ this.bee = bee;
+ }
+
+
+ @Override
+ public Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Received bit com message of type {}", rpcType);
+
+ switch (rpcType) {
+
+ case RpcType.REQ_CANCEL_FRAGMENT_VALUE:
+ FragmentHandle handle = get(pBody, FragmentHandle.PARSER);
+ cancelFragment(handle);
+ return DataRpcConfig.OK;
+
+ case RpcType.REQ_FRAGMENT_STATUS_VALUE:
+ bee.getContext().getWorkBus().status( get(pBody, FragmentStatus.PARSER));
+ // TODO: Support a type of message that has no response.
+ return DataRpcConfig.OK;
+
+ case RpcType.REQ_INIATILIZE_FRAGMENT_VALUE:
+ PlanFragment fragment = get(pBody, PlanFragment.PARSER);
+ try {
+ startNewRemoteFragment(fragment);
+ return DataRpcConfig.OK;
+
+ } catch (OutOfMemoryException e) {
+ logger.error("Failure while attempting to start remote fragment.", fragment);
+ return new Response(RpcType.ACK, Acks.FAIL);
+ }
+
+ default:
+ throw new RpcException("Not yet supported.");
+ }
+
+ }
+
+
+
+ /* (non-Javadoc)
+ * @see org.apache.drill.exec.work.batch.BitComHandler#startNewRemoteFragment(org.apache.drill.exec.proto.ExecProtos.PlanFragment)
+ */
+ @Override
+ public void startNewRemoteFragment(PlanFragment fragment) throws OutOfMemoryException{
+ logger.debug("Received remote fragment start instruction", fragment);
+ FragmentContext context = new FragmentContext(bee.getContext(), fragment, null, new FunctionImplementationRegistry(bee.getContext().getConfig()));
+ ControlTunnel tunnel = bee.getContext().getController().getTunnel(fragment.getForeman());
+
+ NonRootStatusReporter listener = new NonRootStatusReporter(context, tunnel);
+ try{
+ FragmentRoot rootOperator = bee.getContext().getPlanReader().readFragmentOperator(fragment.getFragmentJson());
+ RootExec exec = ImplCreator.getExec(context, rootOperator);
+ FragmentExecutor fr = new FragmentExecutor(context, exec, listener);
+ bee.addFragmentRunner(fr);
+
+ }catch(IOException e){
+ listener.fail(fragment.getHandle(), "Failure while parsing fragment execution plan.", e);
+ }catch(ExecutionSetupException e){
+ listener.fail(fragment.getHandle(), "Failure while setting up execution plan.", e);
+ } catch (Exception e) {
+ listener.fail(fragment.getHandle(), "Failure due to uncaught exception", e);
+ } catch (OutOfMemoryError t) {
+ if(t.getMessage().startsWith("Direct buffer")){
+ listener.fail(fragment.getHandle(), "Failure due to error", t);
+ }else{
+ throw t;
+ }
+
+ }
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.drill.exec.work.batch.BitComHandler#cancelFragment(org.apache.drill.exec.proto.ExecProtos.FragmentHandle)
+ */
+ @Override
+ public Ack cancelFragment(FragmentHandle handle){
+ FragmentManager manager = bee.getContext().getWorkBus().getFragmentManager(handle);
+
+ if(manager != null){
+ // try remote fragment cancel.
+ manager.cancel();
+ }else{
+ // then try local cancel.
+ FragmentExecutor runner = bee.getFragmentRunner(handle);
+ if(runner != null) runner.cancel();
+ }
+
+ return Acks.OK;
+ }
+
+
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
index 116865971..95f2dc680 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
@@ -19,23 +19,23 @@ package org.apache.drill.exec.work.batch;
import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.memory.OutOfMemoryException;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitConnection;
-import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.apache.drill.exec.rpc.control.ControlConnection;
+import org.apache.drill.exec.work.fragment.FragmentManager;
-public interface BitComHandler {
+public interface ControlMessageHandler {
- public abstract Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody)
+ public abstract Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody)
throws RpcException;
- public abstract void startNewRemoteFragment(PlanFragment fragment);
+ public abstract void startNewRemoteFragment(PlanFragment fragment) throws OutOfMemoryException;
public abstract Ack cancelFragment(FragmentHandle handle);
- public abstract void registerIncomingFragmentHandler(IncomingFragmentHandler handler);
} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/DataCollector.java
index 539393ccc..67d78ace2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/DataCollector.java
@@ -18,15 +18,15 @@
package org.apache.drill.exec.work.batch;
-import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
-
import java.io.IOException;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection;
+
-interface BatchCollector {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchCollector.class);
- public boolean batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch) throws IOException ;
+interface DataCollector {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataCollector.class);
+ public boolean batchArrived(int minorFragmentId, RawFragmentBatch batch) throws IOException ;
public int getOppositeMajorFragmentId();
public RawBatchBuffer[] getBuffers();
public int getTotalIncomingFragments();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
index 992d9a976..3cb18b631 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
@@ -18,7 +18,7 @@
package org.apache.drill.exec.work.batch;
import java.io.IOException;
-import java.util.List;
+import java.util.Arrays;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
@@ -28,10 +28,8 @@ import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.physical.base.Receiver;
import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
/**
@@ -42,12 +40,12 @@ public class IncomingBuffers {
private final AtomicInteger streamsRemaining = new AtomicInteger(0);
private final AtomicInteger remainingRequired = new AtomicInteger(0);
- private final Map<Integer, BatchCollector> fragCounts;
+ private final Map<Integer, DataCollector> fragCounts;
private final FragmentContext context;
public IncomingBuffers(PhysicalOperator root, FragmentContext context) {
this.context = context;
- Map<Integer, BatchCollector> counts = Maps.newHashMap();
+ Map<Integer, DataCollector> counts = Maps.newHashMap();
CountRequiredFragments reqFrags = new CountRequiredFragments();
root.accept(reqFrags, counts);
@@ -56,24 +54,24 @@ public class IncomingBuffers {
// Determine the total number of incoming streams that will need to be completed before we are finished.
int totalStreams = 0;
- for(BatchCollector bc : fragCounts.values()){
+ for(DataCollector bc : fragCounts.values()){
totalStreams += bc.getTotalIncomingFragments();
}
assert totalStreams >= remainingRequired.get() : String.format("Total Streams %d should be more than the minimum number of streams to commence (%d). It isn't.", totalStreams, remainingRequired.get());
streamsRemaining.set(totalStreams);
}
- public boolean batchArrived(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {
+ public boolean batchArrived(RawFragmentBatch batch) throws FragmentSetupException {
// no need to do anything if we've already enabled running.
- logger.debug("New Batch Arrived {}", batch);
+// logger.debug("New Batch Arrived {}", batch);
if(batch.getHeader().getIsLastBatch()){
streamsRemaining.decrementAndGet();
}
int sendMajorFragmentId = batch.getHeader().getSendingMajorFragmentId();
- BatchCollector fSet = fragCounts.get(sendMajorFragmentId);
- if (fSet == null) throw new FragmentSetupException(String.format("We received a major fragment id that we were not expecting. The id was %d.", sendMajorFragmentId));
+ DataCollector fSet = fragCounts.get(sendMajorFragmentId);
+ if (fSet == null) throw new FragmentSetupException(String.format("We received a major fragment id that we were not expecting. The id was %d. %s", sendMajorFragmentId, Arrays.toString(fragCounts.values().toArray())));
try {
- boolean decremented = fSet.batchArrived(throttle, batch.getHeader().getSendingMinorFragmentId(), batch);
+ boolean decremented = fSet.batchArrived(batch.getHeader().getSendingMinorFragmentId(), batch);
// we should only return true if remaining required has been decremented and is currently equal to zero.
return decremented && remainingRequired.get() == 0;
@@ -96,11 +94,11 @@ public class IncomingBuffers {
/**
* Designed to setup initial values for arriving fragment accounting.
*/
- public class CountRequiredFragments extends AbstractPhysicalVisitor<Void, Map<Integer, BatchCollector>, RuntimeException> {
+ public class CountRequiredFragments extends AbstractPhysicalVisitor<Void, Map<Integer, DataCollector>, RuntimeException> {
@Override
- public Void visitReceiver(Receiver receiver, Map<Integer, BatchCollector> counts) throws RuntimeException {
- BatchCollector set;
+ public Void visitReceiver(Receiver receiver, Map<Integer, DataCollector> counts) throws RuntimeException {
+ DataCollector set;
if (receiver.supportsOutOfOrderExchange()) {
set = new MergingCollector(remainingRequired, receiver, context);
} else {
@@ -114,7 +112,7 @@ public class IncomingBuffers {
@Override
- public Void visitOp(PhysicalOperator op, Map<Integer, BatchCollector> value) throws RuntimeException {
+ public Void visitOp(PhysicalOperator op, Map<Integer, DataCollector> value) throws RuntimeException {
for(PhysicalOperator o : op){
o.accept(this, value);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
index 1c92bbbba..806b11528 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
@@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.base.Receiver;
-public class MergingCollector extends AbstractFragmentCollector{
+public class MergingCollector extends AbstractDataCollector{
private AtomicInteger streamsRunning;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
index f998eff94..c1f4fa59b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
@@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.base.Receiver;
-public class PartitionedCollector extends AbstractFragmentCollector{
+public class PartitionedCollector extends AbstractDataCollector{
public PartitionedCollector(AtomicInteger parentAccounter, Receiver receiver, FragmentContext context) {
super(parentAccounter, receiver, receiver.getProvidingEndpoints().size(), context);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java
index e7d3d0618..5579658b4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java
@@ -17,19 +17,30 @@
*/
package org.apache.drill.exec.work.batch;
+import java.io.IOException;
+
import org.apache.drill.exec.record.RawFragmentBatch;
import org.apache.drill.exec.record.RawFragmentBatchProvider;
-import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.rpc.RemoteConnection;
-import java.io.IOException;
-
-public interface RawBatchBuffer extends RawFragmentBatchProvider{
+/**
+ * A batch buffer is responsible for queuing incoming batches until a consumer is ready to receive them. It will also
+ * inform upstream if the batch cannot be accepted.
+ */
+public interface RawBatchBuffer extends RawFragmentBatchProvider {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RawBatchBuffer.class);
-
- public void enqueue(ConnectionThrottle throttle, RawFragmentBatch batch) throws IOException;
-
+
+ /**
+ * Add the next new raw fragment batch to the buffer.
+ *
+ * @param batch
+ * Batch to enqueue
+ * @throws IOException
+ */
+ public void enqueue(RawFragmentBatch batch) throws IOException;
+
/**
- * Inform the buffer that no more records are expected.
+ * Inform the buffer that no more batches are expected.
*/
public void finished();
}
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 fa20b3d6a..d7c407bd3 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
@@ -17,18 +17,24 @@
*/
package org.apache.drill.exec.work.batch;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Queues;
import io.netty.buffer.ByteBuf;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
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.BitData;
import org.apache.drill.exec.proto.ExecProtos;
import org.apache.drill.exec.proto.helper.QueryIdHelper;
import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.rpc.RemoteConnection;
import org.apache.drill.exec.store.LocalSyncableFileSystem;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -36,14 +42,9 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Queues;
/**
* This implementation of RawBatchBuffer starts writing incoming batches to disk once the buffer size reaches a threshold.
@@ -85,7 +86,7 @@ public class SpoolingRawBatchBuffer implements RawBatchBuffer {
}
@Override
- public synchronized void enqueue(ConnectionThrottle throttle, RawFragmentBatch batch) throws IOException {
+ public synchronized void enqueue(RawFragmentBatch batch) throws IOException {
RawFragmentBatchWrapper wrapper;
boolean spool = spooling.get();
wrapper = new RawFragmentBatchWrapper(batch, !spool);
@@ -224,10 +225,10 @@ public class SpoolingRawBatchBuffer implements RawBatchBuffer {
public void readFromStream(FSDataInputStream stream) throws IOException {
Stopwatch watch = new Stopwatch();
watch.start();
- ExecProtos.FragmentRecordBatch header = ExecProtos.FragmentRecordBatch.parseDelimitedFrom(stream);
+ BitData.FragmentRecordBatch header = BitData.FragmentRecordBatch.parseDelimitedFrom(stream);
ByteBuf buf = context.getAllocator().buffer(bodyLength);
buf.writeBytes(stream, bodyLength);
- batch = new RawFragmentBatch(header, buf);
+ batch = new RawFragmentBatch(null, header, buf);
available = true;
latch.countDown();
long t = watch.elapsed(TimeUnit.MICROSECONDS);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
index 64012c61a..0175bb67a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
@@ -17,38 +17,46 @@
*/
package org.apache.drill.exec.work.batch;
-import java.util.Iterator;
import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.rpc.RemoteConnection;
import com.google.common.collect.Queues;
public class UnlimitedRawBatchBuffer implements RawBatchBuffer{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnlimitedRawBatchBuffer.class);
- private final LinkedBlockingDeque<RawFragmentBatch> buffer = Queues.newLinkedBlockingDeque();
+ private final LinkedBlockingDeque<RawFragmentBatch> buffer;
private volatile boolean finished = false;
+ private int softlimit;
+ private int startlimit;
+ private AtomicBoolean overlimit;
public UnlimitedRawBatchBuffer(FragmentContext context) {
-
+ softlimit = context.getConfig().getInt(ExecConstants.INCOMING_BUFFER_SIZE);
+ startlimit = softlimit/2;
+ buffer = Queues.newLinkedBlockingDeque();
}
@Override
- public void enqueue(ConnectionThrottle throttle, RawFragmentBatch batch) {
+ public void enqueue(RawFragmentBatch batch) {
buffer.add(batch);
+ if(buffer.size() == softlimit){
+ overlimit.set(true);
+ batch.getConnection().setAutoRead(false);
+ }
}
-// @Override
-// public RawFragmentBatch dequeue() {
-// return buffer.poll();
-// }
-
@Override
public void kill(FragmentContext context) {
- // TODO: Pass back or kill handler?
+ while(!buffer.isEmpty()){
+ RawFragmentBatch batch = buffer.poll();
+ batch.getBody().release();
+ }
}
@@ -60,15 +68,27 @@ public class UnlimitedRawBatchBuffer implements RawBatchBuffer{
@Override
public RawFragmentBatch getNext(){
- RawFragmentBatch b = buffer.poll();
+ RawFragmentBatch b = null;
+
+ b = buffer.poll();
+
+ // if we didn't get a buffer, block on waiting for buffer.
if(b == null && !finished){
try {
- return buffer.take();
+ b = buffer.take();
} catch (InterruptedException e) {
return null;
}
}
+ // if we are in the overlimit condition and aren't finished, check if we've passed the start limit. If so, turn off the overlimit condition and set auto read to true (start reading from socket again).
+ if(!finished && overlimit.get()){
+ if(buffer.size() == startlimit){
+ overlimit.set(false);
+ b.getConnection().setAutoRead(true);
+ }
+ }
+
return b;
}
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 726539a63..e4f8e7b9c 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
@@ -37,7 +37,7 @@ import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
import org.apache.drill.exec.planner.fragment.PlanningSet;
import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
import org.apache.drill.exec.planner.fragment.StatsCollector;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
@@ -50,10 +50,10 @@ import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.util.AtomicState;
+import org.apache.drill.exec.work.ErrorHelper;
import org.apache.drill.exec.work.QueryWorkUnit;
import org.apache.drill.exec.work.WorkManager.WorkerBee;
-import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
/**
@@ -65,19 +65,18 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
private QueryId queryId;
private RunQuery queryRequest;
private QueryContext context;
- private RunningFragmentManager fragmentManager;
+ private QueryManager fragmentManager;
private WorkerBee bee;
private UserClientConnection initiatingClient;
private final AtomicState<QueryState> state;
-
public Foreman(WorkerBee bee, DrillbitContext dContext, UserClientConnection connection, QueryId queryId,
RunQuery queryRequest) {
this.queryId = queryId;
this.queryRequest = queryRequest;
this.context = new QueryContext(queryId, dContext);
this.initiatingClient = connection;
- this.fragmentManager = new RunningFragmentManager(new ForemanManagerListener(), new TunnelManager(dContext.getBitCom()));
+ this.fragmentManager = new QueryManager(new ForemanManagerListener(), dContext.getController());
this.bee = bee;
this.state = new AtomicState<QueryState>(QueryState.PENDING) {
@@ -214,17 +213,19 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
context.getPlanReader(), rootFragment, planningSet, context.getConfig().getInt(ExecConstants.GLOBAL_MAX_WIDTH),
context.getConfig().getInt(ExecConstants.MAX_WIDTH_PER_ENDPOINT));
- this.context.getBitCom().getListeners().addFragmentStatusListener(work.getRootFragment().getHandle(), fragmentManager);
+ this.context.getWorkBus().setFragmentStatusListener(work.getRootFragment().getHandle().getQueryId(), fragmentManager);
List<PlanFragment> leafFragments = Lists.newArrayList();
List<PlanFragment> intermediateFragments = Lists.newArrayList();
// store fragments in distributed grid.
for (PlanFragment f : work.getFragments()) {
+
+ // store all fragments in grid since they are part of handshake.
+ context.getCache().storeFragment(f);
if (f.getLeafFragment()) {
leafFragments.add(f);
} else {
intermediateFragments.add(f);
- context.getCache().storeFragment(f);
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java
index 2fbb70804..afc8d796e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java
@@ -17,7 +17,7 @@
*/
package org.apache.drill.exec.work.foreman;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
public interface FragmentStatusListener {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStatusListener.class);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index 2cb57dde1..e9302e1fc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -29,45 +29,48 @@ import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.base.FragmentRoot;
import org.apache.drill.exec.physical.impl.ImplCreator;
import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.proto.UserProtos.QueryResult;
import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.rpc.control.Controller;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
-import org.apache.drill.exec.work.AbstractFragmentRunnerListener;
import org.apache.drill.exec.work.EndpointListener;
-import org.apache.drill.exec.work.FragmentRunner;
import org.apache.drill.exec.work.WorkManager.WorkerBee;
import org.apache.drill.exec.work.batch.IncomingBuffers;
import org.apache.drill.exec.work.foreman.Foreman.ForemanManagerListener;
-import org.apache.drill.exec.work.fragment.LocalFragmentHandler;
+import org.apache.drill.exec.work.fragment.AbstractStatusReporter;
+import org.apache.drill.exec.work.fragment.FragmentExecutor;
+import org.apache.drill.exec.work.fragment.RootFragmentManager;
import com.google.common.collect.Maps;
/**
* Each Foreman holds its own fragment manager. This manages the events associated with execution of a particular query across all fragments.
*/
-class RunningFragmentManager implements FragmentStatusListener{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RunningFragmentManager.class);
+class QueryManager implements FragmentStatusListener{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryManager.class);
- public Map<FragmentHandle, FragmentData> map = Maps.newHashMap(); // doesn't need to be
- private final TunnelManager tun;
+ public Map<FragmentHandle, FragmentData> map = Maps.newHashMap(); // doesn't need to be thread safe as map is generated in a single thread and then accessed by multiple threads for reads only.
+ private final Controller controller;
private ForemanManagerListener foreman;
private AtomicInteger remainingFragmentCount;
- private FragmentRunner rootRunner;
+ private WorkEventBus workBus;
+ private FragmentExecutor rootRunner;
private volatile QueryId queryId;
- public RunningFragmentManager(ForemanManagerListener foreman, TunnelManager tun) {
+ public QueryManager(ForemanManagerListener foreman, Controller controller) {
super();
this.foreman = foreman;
- this.tun = tun;
+ this.controller = controller;
this.remainingFragmentCount = new AtomicInteger(0);
}
@@ -75,21 +78,25 @@ class RunningFragmentManager implements FragmentStatusListener{
public void runFragments(WorkerBee bee, PlanFragment rootFragment, FragmentRoot rootOperator, UserClientConnection rootClient, List<PlanFragment> leafFragments, List<PlanFragment> intermediateFragments) throws ExecutionSetupException{
remainingFragmentCount.set(leafFragments.size()+1);
queryId = rootFragment.getHandle().getQueryId();
+ workBus = bee.getContext().getWorkBus();
// set up the root fragment first so we'll have incoming buffers available.
{
- FragmentContext rootContext = new FragmentContext(bee.getContext(), rootFragment.getHandle(), rootClient, new FunctionImplementationRegistry(bee.getContext().getConfig()));
+ FragmentContext rootContext = new FragmentContext(bee.getContext(), rootFragment, rootClient, new FunctionImplementationRegistry(bee.getContext().getConfig()));
IncomingBuffers buffers = new IncomingBuffers(rootOperator, rootContext);
rootContext.setBuffers(buffers);
RootExec rootExec = ImplCreator.getExec(rootContext, rootOperator);
// add fragment to local node.
map.put(rootFragment.getHandle(), new FragmentData(rootFragment.getHandle(), null, true));
- rootRunner = new FragmentRunner(rootContext, rootExec, new RootFragmentManager(rootContext, rootFragment));
- LocalFragmentHandler handler = new LocalFragmentHandler(rootFragment.getHandle(), buffers, rootRunner);
+ rootRunner = new FragmentExecutor(rootContext, rootExec, new RootStatusHandler(rootContext, rootFragment));
+ RootFragmentManager fragmentManager = new RootFragmentManager(rootFragment.getHandle(), buffers, rootRunner);
+
if(buffers.isDone()){
- bee.addFragmentRunner(handler.getRunnable());
+ // if we don't have to wait for any incoming data, start the fragment runner.
+ bee.addFragmentRunner(fragmentManager.getRunnable());
}else{
- bee.getContext().getBitCom().registerIncomingBatchHandler(handler);
+ // if we do, record the fragment manager in the workBus.
+ workBus.setRootFragmentManager(fragmentManager);
}
}
@@ -111,7 +118,7 @@ class RunningFragmentManager implements FragmentStatusListener{
logger.debug("Sending remote fragment to node {} with data {}", fragment.getAssignment(), fragment.getFragmentJson());
map.put(fragment.getHandle(), new FragmentData(fragment.getHandle(), fragment.getAssignment(), false));
FragmentSubmitListener listener = new FragmentSubmitListener(fragment.getAssignment(), fragment);
- tun.get(fragment.getAssignment()).sendFragment(listener, fragment);
+ controller.getTunnel(fragment.getAssignment()).sendFragment(listener, fragment);
}
@@ -266,15 +273,15 @@ class RunningFragmentManager implements FragmentStatusListener{
}
- private class RootFragmentManager extends AbstractFragmentRunnerListener{
+ private class RootStatusHandler extends AbstractStatusReporter{
- private RootFragmentManager(FragmentContext context, PlanFragment fragment){
+ private RootStatusHandler(FragmentContext context, PlanFragment fragment){
super(context);
}
@Override
protected void statusChange(FragmentHandle handle, FragmentStatus status) {
- RunningFragmentManager.this.statusUpdate(status);
+ QueryManager.this.statusUpdate(status);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RootStatusReporter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RootStatusReporter.java
new file mode 100644
index 000000000..6c7694a80
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RootStatusReporter.java
@@ -0,0 +1,39 @@
+/**
+ * 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.work.foreman;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.work.fragment.AbstractStatusReporter;
+
+public class RootStatusReporter extends AbstractStatusReporter{
+
+ QueryManager runningFragmentManager;
+
+ private RootStatusReporter(FragmentContext context){
+ super(context);
+ }
+
+ @Override
+ protected void statusChange(FragmentHandle handle, FragmentStatus status) {
+ runningFragmentManager.statusUpdate(status);
+ }
+
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
index 58b6d08f4..105afc2c8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
@@ -15,28 +15,28 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.work;
+package org.apache.drill.exec.work.fragment;
import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
-import org.apache.drill.exec.work.foreman.ErrorHelper;
+import org.apache.drill.exec.work.ErrorHelper;
-public abstract class AbstractFragmentRunnerListener implements FragmentRunnerListener{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractFragmentRunnerListener.class);
+public abstract class AbstractStatusReporter implements StatusReporter{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStatusReporter.class);
private FragmentContext context;
private volatile long startNanos;
- public AbstractFragmentRunnerListener(FragmentContext context) {
+ public AbstractStatusReporter(FragmentContext context) {
super();
this.context = context;
}
private FragmentStatus.Builder getBuilder(FragmentState state){
FragmentStatus.Builder status = FragmentStatus.newBuilder();
- context.addMetricsToStatus(status);
+ context.getStats().addMetricsToStatus(status);
status.setState(state);
status.setRunningTime(System.nanoTime() - startNanos);
status.setHandle(context.getHandle());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
index d00397223..8bb8ae14b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
@@ -15,31 +15,33 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.work;
+package org.apache.drill.exec.work.fragment;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.impl.RootExec;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.work.CancelableQuery;
+import org.apache.drill.exec.work.StatusProvider;
import com.codahale.metrics.Timer;
/**
* Responsible for running a single fragment on a single Drillbit. Listens/responds to status request and cancellation
- * messages. Two child implementation, root (driving) and child (driven) exist.
+ * messages.
*/
-public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider, Comparable<Object>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRunner.class);
+public class FragmentExecutor implements Runnable, CancelableQuery, StatusProvider, Comparable<Object>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentExecutor.class);
private final AtomicInteger state = new AtomicInteger(FragmentState.AWAITING_ALLOCATION_VALUE);
private final RootExec root;
private final FragmentContext context;
- private final FragmentRunnerListener listener;
+ private final StatusReporter listener;
- public FragmentRunner(FragmentContext context, RootExec root, FragmentRunnerListener listener){
+ public FragmentExecutor(FragmentContext context, RootExec root, StatusReporter listener){
this.context = context;
this.root = root;
this.listener = listener;
@@ -48,8 +50,8 @@ public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider
@Override
public FragmentStatus getStatus() {
return FragmentStatus.newBuilder() //
- .setBatchesCompleted(context.batchesCompleted.get()) //
- .setDataProcessed(context.dataProcessed.get()) //
+ .setBatchesCompleted(context.getStats().batchesCompleted.get()) //
+ .setDataProcessed(context.getStats().dataProcessed.get()) //
.setMemoryUse(context.getAllocator().getAllocatedMemory()) //
.build();
}
@@ -65,13 +67,15 @@ public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider
@Override
public void run() {
+
+ boolean closed = false;
logger.debug("Starting fragment runner. {}:{}", context.getHandle().getMajorFragmentId(), context.getHandle().getMinorFragmentId());
if(!updateState(FragmentState.AWAITING_ALLOCATION, FragmentState.RUNNING, false)){
internalFail(new RuntimeException(String.format("Run was called when fragment was in %s state. FragmentRunnables should only be started when they are currently in awaiting allocation state.", FragmentState.valueOf(state.get()))));
return;
}
- Timer.Context t = context.fragmentTime.time();
+ Timer.Context t = context.getStats().fragmentTime.time();
// run the query until root.next returns false.
try{
@@ -86,16 +90,21 @@ public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider
}
}
- // If this isn't a finished stop, we'll inform other batches to finish up.
- if(state.get() != FragmentState.FINISHED_VALUE){
- root.stop();
- }
+ root.stop();
+
+ closed = true;
+ context.close();
}catch(Exception ex){
logger.debug("Caught exception while running fragment", ex);
internalFail(ex);
}finally{
t.stop();
+ if(!closed) try{
+ context.close();
+ }catch(RuntimeException e){
+ logger.warn("Failure while closing context in failed state.", e);
+ }
}
logger.debug("Fragment runner complete. {}:{}", context.getHandle().getMajorFragmentId(), context.getHandle().getMinorFragmentId());
}
@@ -127,5 +136,8 @@ public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider
return o.hashCode() - this.hashCode();
}
+ public FragmentContext getContext(){
+ return context;
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
index a29c600b3..7d92c9a64 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
@@ -18,32 +18,37 @@
package org.apache.drill.exec.work.fragment;
import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
-import org.apache.drill.exec.work.FragmentRunner;
/**
- * Handles incoming fragments as they arrive, routing them as apporpriate.
+ * The Fragment Manager is responsible managing incoming data and executing a fragment. Once enough data and resources
+ * are avialable, a fragment manager will start a fragment executor to run the associated fragment.
*/
-public interface IncomingFragmentHandler {
+public interface FragmentManager {
/**
- * Handle the next incoming fragment.
- * @param throttle
+ * Handle the next incoming record batch.
+ *
* @param batch
* @return True if the fragment has enough incoming data to be able to be run.
* @throws FragmentSetupException
*/
- public abstract boolean handle(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException;
+ public abstract boolean handle(RawFragmentBatch batch) throws FragmentSetupException;
/**
- * Get the fragment runner for this incoming fragment. Note, this can only be requested once.
+ * Get the fragment runner for this incoming fragment. Note, this can only be requested once.
+ *
* @return
*/
- public abstract FragmentRunner getRunnable();
+ public abstract FragmentExecutor getRunnable();
public abstract void cancel();
+
public boolean isWaiting();
+
public abstract FragmentHandle getHandle();
+
+ public abstract FragmentContext getFragmentContext();
} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
index 615722984..d82c1c001 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
@@ -28,38 +28,33 @@ import org.apache.drill.exec.physical.base.FragmentRoot;
import org.apache.drill.exec.physical.impl.ImplCreator;
import org.apache.drill.exec.physical.impl.RootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
-import org.apache.drill.exec.rpc.bit.BitTunnel;
import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.work.FragmentRunner;
-import org.apache.drill.exec.work.FragmentRunnerListener;
-import org.apache.drill.exec.work.RemoteFragmentRunnerListener;
import org.apache.drill.exec.work.batch.IncomingBuffers;
/**
- * This handler receives all incoming traffic for a particular FragmentHandle. It will monitor the state of incoming batches
+ * This managers determines when to run a non-root fragment node.
*/
-public class RemoteFragmentHandler implements IncomingFragmentHandler {
+public class NonRootFragmentManager implements FragmentManager {
private final PlanFragment fragment;
private FragmentLeaf root;
private final IncomingBuffers buffers;
- private final FragmentRunnerListener runnerListener;
- private volatile FragmentRunner runner;
+ private final StatusReporter runnerListener;
+ private volatile FragmentExecutor runner;
private volatile boolean cancel = false;
private final FragmentContext context;
private final PhysicalPlanReader reader;
- public RemoteFragmentHandler(PlanFragment fragment, DrillbitContext context, BitTunnel foremanTunnel) throws FragmentSetupException{
+ public NonRootFragmentManager(PlanFragment fragment, DrillbitContext context) throws FragmentSetupException{
try{
this.fragment = fragment;
this.root = context.getPlanReader().readFragmentOperator(fragment.getFragmentJson());
- this.context = new FragmentContext(context, fragment.getHandle(), null, new FunctionImplementationRegistry(context.getConfig()));
+ this.context = new FragmentContext(context, fragment, null, new FunctionImplementationRegistry(context.getConfig()));
this.buffers = new IncomingBuffers(root, this.context);
this.context.setBuffers(buffers);
- this.runnerListener = new RemoteFragmentRunnerListener(this.context, foremanTunnel);
+ this.runnerListener = new NonRootStatusReporter(this.context, context.getController().getTunnel(fragment.getForeman()));
this.reader = context.getPlanReader();
}catch(IOException e){
@@ -71,22 +66,22 @@ public class RemoteFragmentHandler implements IncomingFragmentHandler {
* @see org.apache.drill.exec.work.fragment.FragmentHandler#handle(org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle, org.apache.drill.exec.record.RawFragmentBatch)
*/
@Override
- public boolean handle(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {
- return buffers.batchArrived(throttle, batch);
+ public boolean handle(RawFragmentBatch batch) throws FragmentSetupException {
+ return buffers.batchArrived(batch);
}
/* (non-Javadoc)
* @see org.apache.drill.exec.work.fragment.FragmentHandler#getRunnable()
*/
@Override
- public FragmentRunner getRunnable(){
+ public FragmentExecutor getRunnable(){
synchronized(this){
if(runner != null) throw new IllegalStateException("Get Runnable can only be run once.");
if(cancel) return null;
try {
FragmentRoot fragRoot = reader.readFragmentOperator(fragment.getFragmentJson());
RootExec exec = ImplCreator.getExec(context, fragRoot);
- runner = new FragmentRunner(context, exec, runnerListener);
+ runner = new FragmentExecutor(context, exec, runnerListener);
return this.runner;
} catch (IOException | ExecutionSetupException e) {
runnerListener.fail(fragment.getHandle(), "Failure while setting up remote fragment.", e);
@@ -118,6 +113,11 @@ public class RemoteFragmentHandler implements IncomingFragmentHandler {
public boolean isWaiting() {
return !buffers.isDone() && !cancel;
}
+
+ @Override
+ public FragmentContext getFragmentContext() {
+ return context;
+ }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootStatusReporter.java
index da8139953..353450039 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemoteFragmentRunnerListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootStatusReporter.java
@@ -15,27 +15,26 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.work;
+package org.apache.drill.exec.work.fragment;
import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.rpc.control.ControlTunnel;
/**
- * Informs remote node as fragment changes state.
+ * For all non root fragments, status will be reported back to the foreman through a control tunnel.
*/
-public class RemoteFragmentRunnerListener extends AbstractFragmentRunnerListener{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteFragmentRunnerListener.class);
+public class NonRootStatusReporter extends AbstractStatusReporter{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NonRootStatusReporter.class);
- private final BitTunnel tunnel;
+ private final ControlTunnel tunnel;
- public RemoteFragmentRunnerListener(FragmentContext context, BitTunnel tunnel) {
+ public NonRootStatusReporter(FragmentContext context, ControlTunnel tunnel) {
super(context);
this.tunnel = tunnel;
}
-
@Override
protected void statusChange(FragmentHandle handle, FragmentStatus status) {
logger.debug("Sending status change message message to remote node: " + status);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
index 12a750129..40f4d2b30 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
@@ -18,21 +18,20 @@
package org.apache.drill.exec.work.fragment;
import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
-import org.apache.drill.exec.work.FragmentRunner;
import org.apache.drill.exec.work.batch.IncomingBuffers;
-public class LocalFragmentHandler implements IncomingFragmentHandler{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalFragmentHandler.class);
+public class RootFragmentManager implements FragmentManager{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootFragmentManager.class);
private final IncomingBuffers buffers;
- private final FragmentRunner runner;
+ private final FragmentExecutor runner;
private final FragmentHandle handle;
private volatile boolean cancel = false;
- public LocalFragmentHandler(FragmentHandle handle, IncomingBuffers buffers, FragmentRunner runner) {
+ public RootFragmentManager(FragmentHandle handle, IncomingBuffers buffers, FragmentExecutor runner) {
super();
this.handle = handle;
this.buffers = buffers;
@@ -40,16 +39,15 @@ public class LocalFragmentHandler implements IncomingFragmentHandler{
}
@Override
- public boolean handle(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {
- return buffers.batchArrived(throttle, batch);
+ public boolean handle(RawFragmentBatch batch) throws FragmentSetupException {
+ return buffers.batchArrived(batch);
}
@Override
- public FragmentRunner getRunnable() {
+ public FragmentExecutor getRunnable() {
return runner;
}
-
public FragmentHandle getHandle() {
return handle;
}
@@ -63,6 +61,11 @@ public class LocalFragmentHandler implements IncomingFragmentHandler{
public boolean isWaiting() {
return !buffers.isDone() && !cancel;
}
+
+ @Override
+ public FragmentContext getFragmentContext() {
+ return runner.getContext();
+ }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
index 918873544..45c1f5e5d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
@@ -15,12 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.work;
+package org.apache.drill.exec.work.fragment;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
-public interface FragmentRunnerListener {
+/**
+ * The status handler is responsible for receiving changes in fragment status and propagating them back to the foreman.
+ */
+public interface StatusReporter {
void fail(FragmentHandle handle, String message, Throwable excep);
void stateChanged(FragmentHandle handle, FragmentState newState);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
index 6f2d85a67..92062050f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
@@ -28,9 +28,9 @@ import org.apache.drill.exec.proto.UserProtos.RequestResults;
import org.apache.drill.exec.proto.UserProtos.RunQuery;
import org.apache.drill.exec.rpc.Acks;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
-import org.apache.drill.exec.work.FragmentRunner;
import org.apache.drill.exec.work.WorkManager.WorkerBee;
import org.apache.drill.exec.work.foreman.Foreman;
+import org.apache.drill.exec.work.fragment.FragmentExecutor;
public class UserWorker{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserWorker.class);
@@ -65,7 +65,7 @@ public class UserWorker{
}
public Ack cancelFragment(FragmentHandle handle){
- FragmentRunner runner = bee.getFragmentRunner(handle);
+ FragmentExecutor runner = bee.getFragmentRunner(handle);
if(runner != null) runner.cancel();
return Acks.OK;
}
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index b84c40693..c801163ac 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -75,9 +75,12 @@ drill.exec: {
directories: ["/tmp/drill"],
filesystem: "drill-local:///"
},
- spooling: {
+ buffer:{
impl: "org.apache.drill.exec.work.batch.SpoolingRawBatchBuffer",
- delete: false,
- size: 100000000
+ size: "20000",
+ spooling: {
+ delete: false,
+ size: 100000000
+ }
}
} \ No newline at end of file
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 c06818da4..c513cd0b3 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
@@ -28,14 +28,15 @@ import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.util.FileUtils;
import org.apache.drill.exec.ExecConstants;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
-import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.helper.QueryIdHelper;
@@ -47,9 +48,9 @@ import org.apache.hadoop.fs.Path;
import org.junit.AfterClass;
import org.junit.Test;
+import com.codahale.metrics.MetricRegistry;
import com.google.common.base.Charsets;
import com.google.common.io.Files;
-import com.codahale.metrics.MetricRegistry;
/**
* The unit test case will read a physical plan in json format. The physical plan contains a "trace" operator,
@@ -66,7 +67,7 @@ public class DumpCatTest {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -74,7 +75,7 @@ public class DumpCatTest {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/trace/simple_trace.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
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 8f5646429..11b6890f3 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
@@ -17,7 +17,8 @@
*/
package org.apache.drill.exec.fn.impl;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
import mockit.Injectable;
import mockit.NonStrictExpectations;
@@ -26,16 +27,16 @@ import org.apache.drill.common.expression.ExpressionPosition;
import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.util.FileUtils;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
-import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.vector.BitVector;
@@ -43,9 +44,9 @@ import org.apache.drill.exec.vector.IntVector;
import org.junit.AfterClass;
import org.junit.Test;
+import com.codahale.metrics.MetricRegistry;
import com.google.common.base.Charsets;
import com.google.common.io.Files;
-import com.codahale.metrics.MetricRegistry;
public class TestRepeatedFunction {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestRepeatedFunction.class);
@@ -59,7 +60,7 @@ public class TestRepeatedFunction {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -67,7 +68,7 @@ public class TestRepeatedFunction {
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);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
boolean oneIsOne = false;
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 885de4935..8e53f3053 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,7 +20,7 @@ package org.apache.drill.exec.memory;
import static org.junit.Assert.assertEquals;
import io.netty.buffer.ByteBuf;
-import org.apache.drill.exec.memory.DirectBufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
import org.junit.Test;
@@ -29,7 +29,7 @@ public class TestEndianess {
@Test
public void testLittleEndian(){
- DirectBufferAllocator a = new DirectBufferAllocator();
+ TopLevelAllocator a = new TopLevelAllocator(Long.MAX_VALUE);
ByteBuf b = a.buffer(4);
b.setInt(0, 35);
assertEquals((int) b.getByte(0), 35);
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 0312863b9..ccf775808 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
@@ -67,7 +67,14 @@ public class SimpleRootExec implements RootExec, Iterable<ValueVector>{
@Override
public boolean next() {
- return incoming.next() != IterOutcome.NONE;
+ switch(incoming.next()){
+ case NONE:
+ case STOP:
+ incoming.cleanup();
+ return false;
+ default:
+ return true;
+ }
}
@Override
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 e51bee74b..3010c37e6 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
@@ -33,11 +33,12 @@ import org.apache.drill.exec.expr.holders.Float8Holder;
import org.apache.drill.exec.expr.holders.IntHolder;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.rpc.user.UserServer;
@@ -49,12 +50,11 @@ import org.apache.drill.exec.vector.IntVector;
import org.apache.drill.exec.vector.VarBinaryVector;
import org.apache.drill.exec.vector.VarCharVector;
import org.junit.After;
-import org.junit.Ignore;
import org.junit.Test;
+import com.codahale.metrics.MetricRegistry;
import com.google.common.base.Charsets;
import com.google.common.io.Files;
-import com.codahale.metrics.MetricRegistry;
public class TestCastFunctions {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleFunctions.class);
@@ -68,7 +68,7 @@ public class TestCastFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -76,7 +76,7 @@ public class TestCastFunctions {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastBigInt.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -108,7 +108,7 @@ public class TestCastFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -116,7 +116,7 @@ public class TestCastFunctions {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastInt.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -148,7 +148,7 @@ public class TestCastFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -156,7 +156,7 @@ public class TestCastFunctions {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastFloat4.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -188,7 +188,7 @@ public class TestCastFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -196,7 +196,7 @@ public class TestCastFunctions {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastFloat8.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -228,7 +228,7 @@ public class TestCastFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -236,7 +236,7 @@ public class TestCastFunctions {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarChar.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -268,7 +268,7 @@ public class TestCastFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -276,7 +276,7 @@ public class TestCastFunctions {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarBinary.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -308,7 +308,7 @@ public class TestCastFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -316,7 +316,7 @@ public class TestCastFunctions {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastNested.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -348,7 +348,7 @@ public class TestCastFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -356,7 +356,7 @@ public class TestCastFunctions {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastNumException.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
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 ea0ac2c6f..ba4c56131 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
@@ -27,12 +27,14 @@ import mockit.NonStrictExpectations;
import org.apache.drill.common.config.DrillConfig;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.rpc.user.UserServer;
import org.apache.drill.exec.server.DrillbitContext;
import org.junit.AfterClass;
@@ -55,14 +57,14 @@ public class TestComparisonFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
String planString = Resources.toString(Resources.getResource(COMPARISON_TEST_PHYSICAL_PLAN), Charsets.UTF_8).replaceAll("EXPRESSION", expression);
if(reader == null) reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
if(registry == null) registry = new FunctionImplementationRegistry(c);
- if(context == null) context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ if(context == null) context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
PhysicalPlan plan = reader.readPhysicalPlan(planString);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
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 d1e1fe258..a8140b9a6 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
@@ -17,38 +17,29 @@
*/
package org.apache.drill.exec.physical.impl;
-import com.google.common.base.Charsets;
-import com.google.common.io.Resources;
-import com.codahale.metrics.MetricRegistry;
-
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
import mockit.Injectable;
import mockit.NonStrictExpectations;
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.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.expr.holders.BigIntHolder;
-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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos;
-import org.apache.drill.exec.proto.ExecProtos;
import org.apache.drill.exec.rpc.user.UserServer;
import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.vector.BigIntVector;
import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.VarCharVector;
import org.junit.AfterClass;
-import org.junit.Ignore;
import org.junit.Test;
-import org.junit.Assert;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.base.Charsets;
+import com.google.common.io.Resources;
public class TestImplicitCastFunctions {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestImplicitCastFunctions.class);
@@ -77,14 +68,14 @@ public class TestImplicitCastFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
String planString = Resources.toString(Resources.getResource(planPath), Charsets.UTF_8);
if(reader == null) reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
if(registry == null) registry = new FunctionImplementationRegistry(c);
- if(context == null) context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ if(context == null) context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
PhysicalPlan plan = reader.readPhysicalPlan(planString);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
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 99ab3623e..2dd861131 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
@@ -31,6 +31,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.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.ops.QueryContext;
import org.apache.drill.exec.opt.BasicOptimizer;
@@ -38,13 +39,16 @@ import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.proto.UserProtos;
import org.apache.drill.exec.record.RecordBatchLoader;
import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.control.Controller;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
+import org.apache.drill.exec.rpc.data.DataConnectionCreator;
import org.apache.drill.exec.rpc.user.QueryResultBatch;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.BootStrapContext;
@@ -58,67 +62,81 @@ import org.junit.AfterClass;
import org.junit.Ignore;
import org.junit.Test;
+import com.codahale.metrics.MetricRegistry;
import com.google.common.base.Charsets;
import com.google.common.io.Files;
import com.google.common.io.Resources;
-import com.codahale.metrics.MetricRegistry;
@Ignore
public class TestOptiqPlans {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOptiqPlans.class);
DrillConfig c = DrillConfig.create();
-
+
@Test
- public void orderBy(@Injectable final BootStrapContext ctxt, @Injectable UserClientConnection connection, @Injectable ClusterCoordinator coord, @Injectable BitCom com, @Injectable DistributedCache cache) throws Throwable{
- SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_order.json", coord, com, cache);
+ public void orderBy(@Injectable final BootStrapContext ctxt, @Injectable UserClientConnection connection,
+ @Injectable ClusterCoordinator coord, @Injectable DataConnectionCreator com, @Injectable DistributedCache cache,
+ @Injectable Controller controller, @Injectable WorkEventBus workBus) throws Throwable {
+ SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_order.json", coord, com, cache, controller, workBus);
}
-
+
@Test
- public void stringFilter(@Injectable final BootStrapContext ctxt, @Injectable UserClientConnection connection, @Injectable ClusterCoordinator coord, @Injectable BitCom com, @Injectable DistributedCache cache) throws Throwable{
- SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_string_filter.json", coord, com, cache);
+ public void stringFilter(@Injectable final BootStrapContext ctxt, @Injectable UserClientConnection connection,
+ @Injectable ClusterCoordinator coord, @Injectable DataConnectionCreator com, @Injectable DistributedCache cache,
+ @Injectable Controller controller, @Injectable WorkEventBus workBus) throws Throwable {
+ SimpleRootExec exec = doLogicalTest(ctxt, connection, "/logical_string_filter.json", coord, com, cache, controller, workBus);
}
-
+
@Test
- public void groupBy(@Injectable final BootStrapContext bitContext, @Injectable UserClientConnection connection, @Injectable ClusterCoordinator coord, @Injectable BitCom com, @Injectable DistributedCache cache) throws Throwable{
- SimpleRootExec exec = doLogicalTest(bitContext, connection, "/logical_group.json", coord, com, cache);
+ public void groupBy(@Injectable final BootStrapContext bitContext, @Injectable UserClientConnection connection,
+ @Injectable ClusterCoordinator coord, @Injectable DataConnectionCreator com, @Injectable DistributedCache cache,
+ @Injectable Controller controller, @Injectable WorkEventBus workBus) throws Throwable {
+ SimpleRootExec exec = doLogicalTest(bitContext, connection, "/logical_group.json", coord, com, cache, controller, workBus);
}
-
- private SimpleRootExec doLogicalTest(final BootStrapContext context, UserClientConnection connection, String file, ClusterCoordinator coord, BitCom com, DistributedCache cache) throws Exception{
- new NonStrictExpectations(){{
- context.getMetrics(); result = new MetricRegistry();
- context.getAllocator(); result = BufferAllocator.getAllocator(c);
- context.getConfig(); result = c;
- }};
+
+ private SimpleRootExec doLogicalTest(final BootStrapContext context, UserClientConnection connection, String file,
+ ClusterCoordinator coord, DataConnectionCreator com, DistributedCache cache, Controller controller, WorkEventBus workBus) throws Exception {
+ new NonStrictExpectations() {
+ {
+ context.getMetrics();
+ result = new MetricRegistry();
+ context.getAllocator();
+ result = new TopLevelAllocator();
+ context.getConfig();
+ result = c;
+ }
+ };
RemoteServiceSet lss = RemoteServiceSet.getLocalServiceSet();
- DrillbitContext bitContext = new DrillbitContext(DrillbitEndpoint.getDefaultInstance(), context, coord, com, cache);
+ DrillbitContext bitContext = new DrillbitContext(DrillbitEndpoint.getDefaultInstance(), context, coord, controller, com, cache, workBus);
QueryContext qc = new QueryContext(QueryId.getDefaultInstance(), bitContext);
PhysicalPlanReader reader = bitContext.getPlanReader();
LogicalPlan plan = reader.readLogicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
- PhysicalPlan pp = new BasicOptimizer(DrillConfig.create(), qc).optimize(new BasicOptimizer.BasicOptimizationContext(), plan);
-
+ PhysicalPlan pp = new BasicOptimizer(DrillConfig.create(), qc).optimize(
+ new BasicOptimizer.BasicOptimizationContext(), plan);
-
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext fctxt = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
- SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(fctxt, (FragmentRoot) pp.getSortedOperators(false).iterator().next()));
+ FragmentContext fctxt = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+ SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(fctxt, (FragmentRoot) pp.getSortedOperators(false)
+ .iterator().next()));
return exec;
-
+
}
-
+
@Test
public void testFilterPlan() throws Exception {
RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
DrillConfig config = DrillConfig.create();
- try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+ try (Drillbit bit1 = new Drillbit(config, serviceSet);
+ DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
bit1.run();
client.connect();
- List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Resources.toString(Resources.getResource("physical_filter.json"),Charsets.UTF_8));
+ List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+ Resources.toString(Resources.getResource("physical_filter.json"), Charsets.UTF_8));
RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
for (QueryResultBatch b : results) {
System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
loader.load(b.getHeader().getDef(), b.getData());
- for (VectorWrapper vw : loader) {
+ for (VectorWrapper<?> vw : loader) {
System.out.println(vw.getValueVector().getField().getName());
ValueVector vv = vw.getValueVector();
for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
@@ -136,15 +154,17 @@ public class TestOptiqPlans {
RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
DrillConfig config = DrillConfig.create();
- try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+ try (Drillbit bit1 = new Drillbit(config, serviceSet);
+ DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
bit1.run();
client.connect();
- List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Resources.toString(Resources.getResource("physical_join.json"),Charsets.UTF_8));
+ List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+ Resources.toString(Resources.getResource("physical_join.json"), Charsets.UTF_8));
RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
for (QueryResultBatch b : results) {
System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
loader.load(b.getHeader().getDef(), b.getData());
- for (VectorWrapper vw : loader) {
+ for (VectorWrapper<?> vw : loader) {
System.out.println(vw.getValueVector().getField().getName());
ValueVector vv = vw.getValueVector();
for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
@@ -156,53 +176,56 @@ public class TestOptiqPlans {
client.close();
}
}
-
+
@Test
public void testFilterString() throws Exception {
RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
DrillConfig config = DrillConfig.create();
- try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+ try (Drillbit bit1 = new Drillbit(config, serviceSet);
+ DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
bit1.run();
client.connect();
- List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL, Resources.toString(Resources.getResource("logical_string_filter.json"),Charsets.UTF_8));
+ List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL,
+ Resources.toString(Resources.getResource("logical_string_filter.json"), Charsets.UTF_8));
RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
for (QueryResultBatch b : results) {
System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
loader.load(b.getHeader().getDef(), b.getData());
- for (VectorWrapper vw : loader) {
+ for (VectorWrapper<?> vw : loader) {
System.out.println(vw.getValueVector().getField().getName());
ValueVector vv = vw.getValueVector();
for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
Object o = vv.getAccessor().getObject(i);
- if(vv instanceof VarBinaryVector){
+ if (vv instanceof VarBinaryVector) {
VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
VarBinaryHolder vbh = new VarBinaryHolder();
x.get(i, vbh);
System.out.printf("%d..%d", vbh.start, vbh.end);
-
- System.out.println("[" + new String( (byte[]) vv.getAccessor().getObject(i)) + "]");
- }else{
+
+ System.out.println("[" + new String((byte[]) vv.getAccessor().getObject(i)) + "]");
+ } else {
System.out.println(vv.getAccessor().getObject(i));
}
-
+
}
}
}
client.close();
}
}
-
-
+
@Test
public void testLogicalJsonScan() throws Exception {
RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
DrillConfig config = DrillConfig.create();
- try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+ try (Drillbit bit1 = new Drillbit(config, serviceSet);
+ DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
bit1.run();
client.connect();
- List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL, Resources.toString(Resources.getResource("logical_json_scan.json"),Charsets.UTF_8));
+ List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL,
+ Resources.toString(Resources.getResource("logical_json_scan.json"), Charsets.UTF_8));
RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
for (QueryResultBatch b : results) {
System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
@@ -212,33 +235,35 @@ public class TestOptiqPlans {
ValueVector vv = vw.getValueVector();
for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
Object o = vv.getAccessor().getObject(i);
- if(vv instanceof VarBinaryVector){
+ if (vv instanceof VarBinaryVector) {
VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
VarBinaryHolder vbh = new VarBinaryHolder();
x.get(i, vbh);
System.out.printf("%d..%d", vbh.start, vbh.end);
-
- System.out.println("[" + new String( (byte[]) vv.getAccessor().getObject(i)) + "]");
- }else{
+
+ System.out.println("[" + new String((byte[]) vv.getAccessor().getObject(i)) + "]");
+ } else {
System.out.println(vv.getAccessor().getObject(i));
}
-
+
}
}
}
client.close();
}
}
-
+
@Test
public void testOrderVarbinary() throws Exception {
RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
DrillConfig config = DrillConfig.create();
- try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+ try (Drillbit bit1 = new Drillbit(config, serviceSet);
+ DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
bit1.run();
client.connect();
- List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Resources.toString(Resources.getResource("physical_order_varbinary.json"),Charsets.UTF_8));
+ List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+ Resources.toString(Resources.getResource("physical_order_varbinary.json"), Charsets.UTF_8));
RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
for (QueryResultBatch b : results) {
System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
@@ -248,43 +273,51 @@ public class TestOptiqPlans {
ValueVector vv = vw.getValueVector();
for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
Object o = vv.getAccessor().getObject(i);
- if(vv instanceof VarBinaryVector){
+ if (vv instanceof VarBinaryVector) {
VarBinaryVector.Accessor x = ((VarBinaryVector) vv).getAccessor();
VarBinaryHolder vbh = new VarBinaryHolder();
x.get(i, vbh);
System.out.printf("%d..%d", vbh.start, vbh.end);
-
- System.out.println("[" + new String( (byte[]) vv.getAccessor().getObject(i)) + "]");
- }else{
+
+ System.out.println("[" + new String((byte[]) vv.getAccessor().getObject(i)) + "]");
+ } else {
System.out.println(vv.getAccessor().getObject(i));
}
-
+
}
}
}
client.close();
}
}
-
- private SimpleRootExec doPhysicalTest(final DrillbitContext bitContext, UserClientConnection connection, String file) throws Exception{
- new NonStrictExpectations(){{
- bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
- bitContext.getConfig(); result = c;
- }};
-
+
+ private SimpleRootExec doPhysicalTest(final DrillbitContext bitContext, UserClientConnection connection, String file)
+ throws Exception {
+ new NonStrictExpectations() {
+ {
+ bitContext.getMetrics();
+ result = new MetricRegistry();
+ bitContext.getAllocator();
+ result = new TopLevelAllocator();
+ bitContext.getConfig();
+ result = c;
+ }
+ };
+
StorageEngineRegistry reg = new StorageEngineRegistry(bitContext);
-
- PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), reg);
+
+ PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(),
+ CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), reg);
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
- SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
+ SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false)
+ .iterator().next()));
return exec;
}
-
+
@AfterClass
- public static void tearDown() throws Exception{
+ public static void tearDown() throws Exception {
// pause to get logger to catch up.
Thread.sleep(1000);
}
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 d92d9fa08..25daa7a83 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
@@ -30,12 +30,14 @@ import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.rpc.user.UserServer;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.vector.NullableVarBinaryVector;
@@ -58,14 +60,14 @@ public class TestSimpleFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testIsNull.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -86,14 +88,14 @@ public class TestSimpleFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testIsNotNull.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -114,14 +116,14 @@ public class TestSimpleFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testSubstring.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -154,14 +156,14 @@ public class TestSimpleFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testSubstringNegative.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
@@ -194,14 +196,14 @@ public class TestSimpleFunctions {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/testByteSubstring.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
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 89f3292ac..66219e83b 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
@@ -27,6 +27,7 @@ import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.util.FileUtils;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -35,6 +36,7 @@ import org.apache.drill.exec.physical.impl.ImplCreator;
import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
@@ -54,14 +56,14 @@ public class TestAgg {
private SimpleRootExec doTest(final DrillbitContext bitContext, UserClientConnection connection, String file) throws Exception{
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
return exec;
}
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 5429bcffd..e2b561c8a 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
@@ -25,6 +25,7 @@ import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.util.FileUtils;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -33,6 +34,7 @@ import org.apache.drill.exec.physical.impl.ImplCreator;
import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
@@ -56,7 +58,7 @@ public class TestSimpleFilter {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -64,7 +66,7 @@ public class TestSimpleFilter {
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);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
assertEquals(50, exec.getRecordCount());
@@ -81,14 +83,14 @@ public class TestSimpleFilter {
public void testSV4Filter(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/filter/test_sv4.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
int recordCount = 0;
while(exec.next()) {
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 0120c7e7f..493fea6e2 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
@@ -30,6 +30,7 @@ import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.util.FileUtils;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -39,6 +40,7 @@ import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.rpc.user.UserServer;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.StorageEngineRegistry;
@@ -63,14 +65,14 @@ public class TestMergeJoin {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/join/merge_join.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
int totalRecordCount = 0;
@@ -112,7 +114,7 @@ public class TestMergeJoin {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -124,7 +126,7 @@ public class TestMergeJoin {
.replace("#{LEFT_FILE}", FileUtils.getResourceAsFile("/join/merge_single_batch.left.json").toURI().toString())
.replace("#{RIGHT_FILE}", FileUtils.getResourceAsFile("/join/merge_single_batch.right.json").toURI().toString()));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
int totalRecordCount = 0;
@@ -165,7 +167,7 @@ public class TestMergeJoin {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -177,7 +179,7 @@ public class TestMergeJoin {
.replace("#{LEFT_FILE}", FileUtils.getResourceAsFile("/join/merge_single_batch.left.json").toURI().toString())
.replace("#{RIGHT_FILE}", FileUtils.getResourceAsFile("/join/merge_single_batch.right.json").toURI().toString()));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
int totalRecordCount = 0;
@@ -218,7 +220,7 @@ public class TestMergeJoin {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -230,7 +232,7 @@ public class TestMergeJoin {
.replace("#{LEFT_FILE}", FileUtils.getResourceAsFile("/join/merge_multi_batch.left.json").toURI().toString())
.replace("#{RIGHT_FILE}", FileUtils.getResourceAsFile("/join/merge_multi_batch.right.json").toURI().toString()));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
int totalRecordCount = 0;
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 d82c0e8c6..2d74fa938 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
@@ -29,6 +29,7 @@ import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.util.FileUtils;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -38,6 +39,7 @@ import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.record.VectorWrapper;
import org.apache.drill.exec.rpc.user.UserServer;
import org.apache.drill.exec.server.DrillbitContext;
@@ -55,7 +57,7 @@ public class TestSimpleLimit {
public void testLimit(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -66,7 +68,7 @@ public class TestSimpleLimit {
public void testLimitNoEnd(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -77,7 +79,7 @@ public class TestSimpleLimit {
public void testLimitAcrossBatches(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -95,7 +97,7 @@ public class TestSimpleLimit {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/limit/" + testPlan), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
int recordCount = 0;
while(exec.next()){
@@ -114,7 +116,7 @@ public class TestSimpleLimit {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/limit/" + testPlan), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
int recordCount = 0;
long sum = 0;
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 2b46f8032..d1061de42 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
@@ -65,7 +65,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
for (int valueIdx = 0; valueIdx < b.getHeader().getRowCount(); valueIdx++) {
List<Object> row = Lists.newArrayList();
batchLoader.load(b.getHeader().getDef(), b.getData());
- for (VectorWrapper vw : batchLoader)
+ for (VectorWrapper<?> vw : batchLoader)
row.add(vw.getValueVector().getField().getName() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx));
for (Object cell : row) {
if (cell == null) {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
index d35a09e98..1746d564f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
@@ -17,9 +17,10 @@
*/
package org.apache.drill.exec.physical.impl.orderedpartitioner;
-import com.beust.jcommander.internal.Lists;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+
import org.apache.commons.math.stat.descriptive.moment.Mean;
import org.apache.commons.math.stat.descriptive.moment.StandardDeviation;
import org.apache.drill.common.config.DrillConfig;
@@ -34,17 +35,15 @@ import org.apache.drill.exec.rpc.user.QueryResultBatch;
import org.apache.drill.exec.server.BootStrapContext;
import org.apache.drill.exec.server.Drillbit;
import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.util.BatchPrinter;
import org.apache.drill.exec.vector.BigIntVector;
import org.apache.drill.exec.vector.Float8Vector;
import org.apache.drill.exec.vector.IntVector;
import org.junit.Assert;
-import org.junit.Ignore;
import org.junit.Test;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
/**
* Tests the OrderedPartitionExchange Operator
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 fa67e06b5..4d3941eed 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
@@ -28,6 +28,7 @@ import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.util.FileUtils;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -36,6 +37,7 @@ import org.apache.drill.exec.physical.impl.ImplCreator;
import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
@@ -59,7 +61,7 @@ public class TestSimpleProjection {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -67,7 +69,7 @@ public class TestSimpleProjection {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/project/test1.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
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 a5837ed26..c508b21ac 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
@@ -27,6 +27,7 @@ import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.util.FileUtils;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -35,6 +36,7 @@ import org.apache.drill.exec.physical.impl.ImplCreator;
import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
@@ -58,7 +60,7 @@ public class TestSimpleSort {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -66,7 +68,7 @@ public class TestSimpleSort {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/sort/one_key_sort.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
int previousInt = Integer.MIN_VALUE;
@@ -106,7 +108,7 @@ public class TestSimpleSort {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -114,7 +116,7 @@ public class TestSimpleSort {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/sort/two_key_sort.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
int previousInt = Integer.MIN_VALUE;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
index b53d1d364..fa04654c9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/TestSVRemover.java
@@ -26,6 +26,7 @@ import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.util.FileUtils;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -34,6 +35,7 @@ import org.apache.drill.exec.physical.impl.ImplCreator;
import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
@@ -58,7 +60,7 @@ public class TestSVRemover {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -66,7 +68,7 @@ public class TestSVRemover {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/remover/test1.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
int count = exec.getRecordCount();
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 11aae2f01..b7b148cb5 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,6 +26,7 @@ import org.apache.drill.common.util.FileUtils;
import org.apache.drill.exec.ExecConstants;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -34,6 +35,7 @@ import org.apache.drill.exec.physical.impl.ImplCreator;
import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
@@ -62,7 +64,7 @@ public class TestTraceMultiRecordBatch {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -70,7 +72,7 @@ public class TestTraceMultiRecordBatch {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/trace/multi_record_batch_trace.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()) {
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 10ce997cc..f166fade9 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
@@ -27,6 +27,7 @@ import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.cache.VectorAccessibleSerializable;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -35,6 +36,7 @@ import org.apache.drill.exec.physical.impl.ImplCreator;
import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.helper.QueryIdHelper;
import org.apache.drill.exec.record.VectorAccessible;
@@ -76,7 +78,7 @@ public class TestTraceOutputDump {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getConfig(); result = c;
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -84,7 +86,7 @@ public class TestTraceOutputDump {
PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/trace/simple_trace.json"), Charsets.UTF_8));
FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
- FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, registry);
+ FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
while(exec.next()){
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 836f177b1..a57b41fa7 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
@@ -28,6 +28,7 @@ import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.util.FileUtils;
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.ops.FragmentContext;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -37,6 +38,7 @@ import org.apache.drill.exec.physical.impl.SimpleRootExec;
import org.apache.drill.exec.planner.PhysicalPlanReader;
import org.apache.drill.exec.proto.CoordinationProtos;
import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.rpc.user.UserServer;
import org.apache.drill.exec.server.DrillbitContext;
import org.junit.After;
@@ -56,7 +58,7 @@ public class TestSimpleUnion {
new NonStrictExpectations(){{
bitContext.getMetrics(); result = new MetricRegistry();
- bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+ bitContext.getAllocator(); result = new TopLevelAllocator();
bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
}};
@@ -64,7 +66,7 @@ public class TestSimpleUnion {
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, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+ 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};
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java b/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
index 6e0bfa375..37e350e18 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
@@ -17,8 +17,6 @@
*/
package org.apache.drill.exec.pop;
-import static org.junit.Assert.assertEquals;
-
import java.util.List;
import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -26,8 +24,8 @@ import org.apache.drill.exec.planner.fragment.Fragment;
import org.apache.drill.exec.planner.fragment.PlanningSet;
import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.work.QueryWorkUnit;
import org.junit.Test;
@@ -55,7 +53,7 @@ public class TestFragmentChecker extends PopUnitTestBase{
List<DrillbitEndpoint> endpoints = Lists.newArrayList();
DrillbitEndpoint localBit = null;
for(int i =0; i < bitCount; i++){
- DrillbitEndpoint b1 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(1234+i).build();
+ DrillbitEndpoint b1 = DrillbitEndpoint.newBuilder().setAddress("localhost").setControlPort(1234+i).build();
if(i ==0) localBit = b1;
endpoints.add(b1);
}
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 4c459602b..3d072cd50 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
@@ -18,13 +18,17 @@
package org.apache.drill.exec.record.vector;
import com.google.common.collect.Lists;
+
import io.netty.buffer.ByteBuf;
+
+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;
import org.apache.drill.common.types.TypeProtos.*;
import org.apache.drill.common.types.Types;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
import org.apache.drill.exec.record.MaterializedField;
import org.apache.drill.exec.record.RecordBatchLoader;
import org.apache.drill.exec.record.VectorWrapper;
@@ -36,17 +40,11 @@ import java.util.List;
import static org.junit.Assert.assertEquals;
-/**
- * Created with IntelliJ IDEA.
- * User: witwolf
- * Date: 8/30/13
- * Time: 5:00 PM
- */
public class TestLoad {
@Test
public void testLoadValueVector() {
- BufferAllocator allocator = BufferAllocator.getAllocator(null);
+ BufferAllocator allocator = new TopLevelAllocator();
ValueVector fixedV = new IntVector(
MaterializedField.create(new SchemaPath("ints", ExpressionPosition.UNKNOWN), Types.required(MinorType.INT)),
allocator);
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 cc4279dc1..d04e4b3d2 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
@@ -25,7 +25,7 @@ import org.apache.drill.common.types.TypeProtos.DataMode;
import org.apache.drill.common.types.TypeProtos.MajorType;
import org.apache.drill.common.types.TypeProtos.MinorType;
import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.memory.DirectBufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
import org.apache.drill.exec.record.MaterializedField;
import org.apache.drill.exec.vector.BitVector;
@@ -37,7 +37,7 @@ import org.junit.Test;
public class TestValueVector {
- DirectBufferAllocator allocator = new DirectBufferAllocator();
+ TopLevelAllocator allocator = new TopLevelAllocator();
@Test
public void testFixedType() {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
index eaf6f5d15..ca040eb8d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
@@ -17,72 +17,144 @@
*/
package org.apache.drill.exec.server;
+import static org.junit.Assert.assertTrue;
import io.netty.buffer.ByteBuf;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import mockit.Injectable;
+
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.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
+import org.apache.drill.exec.proto.BitData.RpcType;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.record.FragmentWritableBatch;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.RemoteConnection;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitConnection;
-import org.apache.drill.exec.rpc.bit.BitConnectionManager;
-import org.apache.drill.exec.rpc.bit.BitRpcConfig;
-import org.apache.drill.exec.rpc.bit.BitServer;
-import org.apache.drill.exec.rpc.bit.BitTunnel.SendFragmentStatus;
-import org.apache.drill.exec.rpc.bit.ConnectionManagerRegistry;
-import org.apache.drill.exec.rpc.bit.ListenerPool;
-import org.apache.drill.exec.work.batch.BitComHandler;
-import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
+import org.apache.drill.exec.rpc.data.DataConnectionManager;
+import org.apache.drill.exec.rpc.data.DataResponseHandler;
+import org.apache.drill.exec.rpc.data.DataServer;
+import org.apache.drill.exec.rpc.data.DataTunnel;
+import org.apache.drill.exec.vector.Float8Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.fragment.FragmentManager;
import org.junit.Test;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Lists;
+
public class TestBitRpc {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBitRpc.class);
-
+
@Test
- public void testBasicConnectionAndHandshake() throws Exception{
+ public void testConnectionBackpressure(@Injectable WorkerBee bee, @Injectable WorkEventBus workBus) throws Exception {
int port = 1234;
BootStrapContext c = new BootStrapContext(DrillConfig.create());
- final BitComTestHandler handler = new BitComTestHandler();
- final ListenerPool listeners = new ListenerPool(2);
- ConnectionManagerRegistry registry = new ConnectionManagerRegistry(handler, c, listeners);
- BitServer server = new BitServer(handler, c, registry, listeners);
+ BootStrapContext c2 = new BootStrapContext(DrillConfig.create());
+
+ DataResponseHandler drp = new BitComTestHandler();
+ DataServer server = new DataServer(c, workBus, drp);
+
port = server.bind(port);
- DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(port).build();
- registry.setEndpoint(ep);
- for(int i =0; i < 10; i++){
- try(BitConnectionManager cm = new BitConnectionManager(ep, ep, handler, c, listeners)){
- SendFragmentStatus cmd = new SendFragmentStatus(FragmentStatus.getDefaultInstance());
- cm.runCommand(cmd);
- cmd.getFuture().checkedGet();
- }
+ DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setDataPort(port).build();
+ DataConnectionManager manager = new DataConnectionManager(FragmentHandle.getDefaultInstance(), ep, c2);
+ DataTunnel tunnel = new DataTunnel(manager);
+ AtomicLong max = new AtomicLong(0);
+ for (int i = 0; i < 40; i++) {
+ long t1 = System.currentTimeMillis();
+ tunnel.sendRecordBatch(new TimingOutcome(max), new FragmentWritableBatch(false, QueryId.getDefaultInstance(), 1,
+ 1, 1, 1, getRandomBatch(c.getAllocator(), 5000)));
+ System.out.println(System.currentTimeMillis() - t1);
+ // System.out.println("sent.");
}
- System.out.println("connected");
+ System.out.println(String.format("Max time: %d", max.get()));
+ assertTrue(max.get() > 2700);
+ Thread.sleep(5000);
}
-
-
-
- private class BitComTestHandler implements BitComHandler{
- @Override
- public Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return BitRpcConfig.OK;
+ private static WritableBatch getRandomBatch(BufferAllocator allocator, int records) {
+ List<ValueVector> vectors = Lists.newArrayList();
+ for (int i = 0; i < 5; i++) {
+ Float8Vector v = (Float8Vector) TypeHelper.getNewVector(
+ MaterializedField.create(new SchemaPath("a", ExpressionPosition.UNKNOWN), Types.required(MinorType.FLOAT8)),
+ allocator);
+ v.allocateNew(records);
+ v.getMutator().generateTestData();
+ v.getMutator().setValueCount(records);
+ vectors.add(v);
+ }
+ return WritableBatch.getBatchNoHV(records, vectors, false);
+ }
+
+ private class TimingOutcome implements RpcOutcomeListener<Ack> {
+ private AtomicLong max;
+ private Stopwatch watch = new Stopwatch().start();
+
+ public TimingOutcome(AtomicLong max) {
+ super();
+ this.max = max;
}
@Override
- public void startNewRemoteFragment(PlanFragment fragment) {
+ public void failed(RpcException ex) {
+ ex.printStackTrace();
}
@Override
- public Ack cancelFragment(FragmentHandle handle) {
- return null;
+ public void success(Ack value, ByteBuf buffer) {
+ long micros = watch.elapsed(TimeUnit.MILLISECONDS);
+ System.out.println(String.format("Total time to send: %d, start time %d", micros, System.currentTimeMillis() - micros));
+ while (true) {
+ long nowMax = max.get();
+ if (nowMax < micros) {
+ if (max.compareAndSet(nowMax, micros))
+ break;
+ } else {
+ break;
+ }
+ }
}
+ }
+
+ private class BitComTestHandler implements DataResponseHandler {
+
+ int v = 0;
+
@Override
- public void registerIncomingFragmentHandler(IncomingFragmentHandler handler) {
+ public Response handle(RemoteConnection connection, FragmentManager manager, FragmentRecordBatch fragmentBatch, ByteBuf data)
+ throws RpcException {
+ // System.out.println("Received.");
+ try {
+ v++;
+ if (v % 10 == 0) {
+ System.out.println("sleeping.");
+ Thread.sleep(3000);
+ }
+ } catch (InterruptedException e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ }
+ return new Response(RpcType.ACK, Acks.OK);
}
-
+
}
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index 0b134ca48..bffc42765 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -36,7 +36,7 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.common.types.TypeProtos.MinorType;
import org.apache.drill.common.util.FileUtils;
import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.memory.DirectBufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.impl.OutputMutator;
import org.apache.drill.exec.proto.SchemaDefProtos;
@@ -131,7 +131,7 @@ public class JSONRecordReaderTest {
new Expectations() {
{
context.getAllocator();
- returns(new DirectBufferAllocator());
+ returns(new TopLevelAllocator());
}
};
JSONRecordReader jr = new JSONRecordReader(context,
@@ -160,7 +160,7 @@ public class JSONRecordReaderTest {
new Expectations() {
{
context.getAllocator();
- returns(new DirectBufferAllocator());
+ returns(new TopLevelAllocator());
}
};
@@ -201,7 +201,7 @@ public class JSONRecordReaderTest {
new Expectations() {
{
context.getAllocator();
- returns(new DirectBufferAllocator());
+ returns(new TopLevelAllocator());
}
};
@@ -260,7 +260,7 @@ public class JSONRecordReaderTest {
new Expectations() {
{
context.getAllocator();
- returns(new DirectBufferAllocator());
+ returns(new TopLevelAllocator());
}
};
@@ -289,7 +289,7 @@ public class JSONRecordReaderTest {
new Expectations() {
{
context.getAllocator();
- returns(new DirectBufferAllocator());
+ returns(new TopLevelAllocator());
}
};
@@ -322,7 +322,7 @@ public class JSONRecordReaderTest {
new Expectations() {
{
context.getAllocator();
- returns(new DirectBufferAllocator());
+ returns(new TopLevelAllocator());
}
};
diff --git a/exec/java-exec/src/test/resources/logback.xml b/exec/java-exec/src/test/resources/logback.xml
index dd32405c3..b8e0ca2eb 100644
--- a/exec/java-exec/src/test/resources/logback.xml
+++ b/exec/java-exec/src/test/resources/logback.xml
@@ -31,10 +31,15 @@
</encoder>
</appender>
- <logger name="org.apache.drill" additivity="false">
- <level value="debug" />
- <appender-ref ref="SOCKET" />
- </logger>
+ <logger name="org.apache.drill" additivity="false">
+ <level value="debug" />
+ <appender-ref ref="SOCKET" />
+ </logger>
+
+<!-- <logger name="io.netty" additivity="false"> -->
+<!-- <level value="debug" /> -->
+<!-- <appender-ref ref="SOCKET" /> -->
+<!-- </logger> -->
<root>
<level value="error" />
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
new file mode 100644
index 000000000..761f19f58
--- /dev/null
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
@@ -0,0 +1,5438 @@
+/**
+ * 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.
+ */
+// Generated by the protocol buffer compiler. DO NOT EDIT!
+// source: BitControl.proto
+
+package org.apache.drill.exec.proto;
+
+public final class BitControl {
+ private BitControl() {}
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistry registry) {
+ }
+ /**
+ * Protobuf enum {@code exec.bit.control.RpcType}
+ *
+ * <pre>
+ *&#47;/// BitControl RPC ///////
+ * </pre>
+ */
+ public enum RpcType
+ implements com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * <code>HANDSHAKE = 0;</code>
+ */
+ HANDSHAKE(0, 0),
+ /**
+ * <code>ACK = 1;</code>
+ */
+ ACK(1, 1),
+ /**
+ * <code>GOODBYE = 2;</code>
+ */
+ GOODBYE(2, 2),
+ /**
+ * <code>REQ_INIATILIZE_FRAGMENT = 3;</code>
+ *
+ * <pre>
+ * bit requests
+ * </pre>
+ */
+ REQ_INIATILIZE_FRAGMENT(3, 3),
+ /**
+ * <code>REQ_CANCEL_FRAGMENT = 6;</code>
+ *
+ * <pre>
+ * send a cancellation message for a fragment, returns Ack
+ * </pre>
+ */
+ REQ_CANCEL_FRAGMENT(4, 6),
+ /**
+ * <code>REQ_FRAGMENT_STATUS = 7;</code>
+ *
+ * <pre>
+ * get a fragment status, returns FragmentStatus
+ * </pre>
+ */
+ REQ_FRAGMENT_STATUS(5, 7),
+ /**
+ * <code>REQ_BIT_STATUS = 8;</code>
+ *
+ * <pre>
+ * get bit status.
+ * </pre>
+ */
+ REQ_BIT_STATUS(6, 8),
+ /**
+ * <code>RESP_FRAGMENT_HANDLE = 9;</code>
+ *
+ * <pre>
+ * bit responses
+ * </pre>
+ */
+ RESP_FRAGMENT_HANDLE(7, 9),
+ /**
+ * <code>RESP_FRAGMENT_STATUS = 10;</code>
+ */
+ RESP_FRAGMENT_STATUS(8, 10),
+ /**
+ * <code>RESP_BIT_STATUS = 11;</code>
+ */
+ RESP_BIT_STATUS(9, 11),
+ ;
+
+ /**
+ * <code>HANDSHAKE = 0;</code>
+ */
+ public static final int HANDSHAKE_VALUE = 0;
+ /**
+ * <code>ACK = 1;</code>
+ */
+ public static final int ACK_VALUE = 1;
+ /**
+ * <code>GOODBYE = 2;</code>
+ */
+ public static final int GOODBYE_VALUE = 2;
+ /**
+ * <code>REQ_INIATILIZE_FRAGMENT = 3;</code>
+ *
+ * <pre>
+ * bit requests
+ * </pre>
+ */
+ public static final int REQ_INIATILIZE_FRAGMENT_VALUE = 3;
+ /**
+ * <code>REQ_CANCEL_FRAGMENT = 6;</code>
+ *
+ * <pre>
+ * send a cancellation message for a fragment, returns Ack
+ * </pre>
+ */
+ public static final int REQ_CANCEL_FRAGMENT_VALUE = 6;
+ /**
+ * <code>REQ_FRAGMENT_STATUS = 7;</code>
+ *
+ * <pre>
+ * get a fragment status, returns FragmentStatus
+ * </pre>
+ */
+ public static final int REQ_FRAGMENT_STATUS_VALUE = 7;
+ /**
+ * <code>REQ_BIT_STATUS = 8;</code>
+ *
+ * <pre>
+ * get bit status.
+ * </pre>
+ */
+ public static final int REQ_BIT_STATUS_VALUE = 8;
+ /**
+ * <code>RESP_FRAGMENT_HANDLE = 9;</code>
+ *
+ * <pre>
+ * bit responses
+ * </pre>
+ */
+ public static final int RESP_FRAGMENT_HANDLE_VALUE = 9;
+ /**
+ * <code>RESP_FRAGMENT_STATUS = 10;</code>
+ */
+ public static final int RESP_FRAGMENT_STATUS_VALUE = 10;
+ /**
+ * <code>RESP_BIT_STATUS = 11;</code>
+ */
+ public static final int RESP_BIT_STATUS_VALUE = 11;
+
+
+ public final int getNumber() { return value; }
+
+ public static RpcType valueOf(int value) {
+ switch (value) {
+ case 0: return HANDSHAKE;
+ case 1: return ACK;
+ case 2: return GOODBYE;
+ case 3: return REQ_INIATILIZE_FRAGMENT;
+ case 6: return REQ_CANCEL_FRAGMENT;
+ case 7: return REQ_FRAGMENT_STATUS;
+ case 8: return REQ_BIT_STATUS;
+ case 9: return RESP_FRAGMENT_HANDLE;
+ case 10: return RESP_FRAGMENT_STATUS;
+ case 11: return RESP_BIT_STATUS;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap<RpcType>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap<RpcType>
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap<RpcType>() {
+ public RpcType findValueByNumber(int number) {
+ return RpcType.valueOf(number);
+ }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(index);
+ }
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.getDescriptor().getEnumTypes().get(0);
+ }
+
+ private static final RpcType[] VALUES = values();
+
+ public static RpcType valueOf(
+ com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+
+ private final int index;
+ private final int value;
+
+ private RpcType(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:exec.bit.control.RpcType)
+ }
+
+ public interface BitControlHandshakeOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // optional int32 rpc_version = 1;
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ boolean hasRpcVersion();
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ int getRpcVersion();
+
+ // optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];</code>
+ */
+ boolean hasChannel();
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];</code>
+ */
+ org.apache.drill.exec.proto.UserBitShared.RpcChannel getChannel();
+
+ // optional .exec.DrillbitEndpoint endpoint = 3;
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ boolean hasEndpoint();
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint();
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder();
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.BitControlHandshake}
+ */
+ public static final class BitControlHandshake extends
+ com.google.protobuf.GeneratedMessage
+ implements BitControlHandshakeOrBuilder {
+ // Use BitControlHandshake.newBuilder() to construct.
+ private BitControlHandshake(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private BitControlHandshake(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final BitControlHandshake defaultInstance;
+ public static BitControlHandshake getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public BitControlHandshake getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private BitControlHandshake(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 8: {
+ bitField0_ |= 0x00000001;
+ rpcVersion_ = input.readInt32();
+ break;
+ }
+ case 16: {
+ int rawValue = input.readEnum();
+ org.apache.drill.exec.proto.UserBitShared.RpcChannel value = org.apache.drill.exec.proto.UserBitShared.RpcChannel.valueOf(rawValue);
+ if (value == null) {
+ unknownFields.mergeVarintField(2, rawValue);
+ } else {
+ bitField0_ |= 0x00000002;
+ channel_ = value;
+ }
+ break;
+ }
+ case 26: {
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ subBuilder = endpoint_.toBuilder();
+ }
+ endpoint_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(endpoint_);
+ endpoint_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000004;
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitControlHandshake_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitControlHandshake_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.BitControlHandshake.class, org.apache.drill.exec.proto.BitControl.BitControlHandshake.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<BitControlHandshake> PARSER =
+ new com.google.protobuf.AbstractParser<BitControlHandshake>() {
+ public BitControlHandshake parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new BitControlHandshake(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<BitControlHandshake> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // optional int32 rpc_version = 1;
+ public static final int RPC_VERSION_FIELD_NUMBER = 1;
+ private int rpcVersion_;
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public boolean hasRpcVersion() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public int getRpcVersion() {
+ return rpcVersion_;
+ }
+
+ // optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];
+ public static final int CHANNEL_FIELD_NUMBER = 2;
+ private org.apache.drill.exec.proto.UserBitShared.RpcChannel channel_;
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];</code>
+ */
+ public boolean hasChannel() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RpcChannel getChannel() {
+ return channel_;
+ }
+
+ // optional .exec.DrillbitEndpoint endpoint = 3;
+ public static final int ENDPOINT_FIELD_NUMBER = 3;
+ private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_;
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public boolean hasEndpoint() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
+ return endpoint_;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
+ return endpoint_;
+ }
+
+ private void initFields() {
+ rpcVersion_ = 0;
+ channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.BIT_CONTROL;
+ endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeInt32(1, rpcVersion_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeEnum(2, channel_.getNumber());
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeMessage(3, endpoint_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt32Size(1, rpcVersion_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(2, channel_.getNumber());
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(3, endpoint_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitControlHandshake parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.drill.exec.proto.BitControl.BitControlHandshake prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.BitControlHandshake}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.drill.exec.proto.BitControl.BitControlHandshakeOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitControlHandshake_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitControlHandshake_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.BitControlHandshake.class, org.apache.drill.exec.proto.BitControl.BitControlHandshake.Builder.class);
+ }
+
+ // Construct using org.apache.drill.exec.proto.BitControl.BitControlHandshake.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getEndpointFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ rpcVersion_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.BIT_CONTROL;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ if (endpointBuilder_ == null) {
+ endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ } else {
+ endpointBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000004);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitControlHandshake_descriptor;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.BitControlHandshake getDefaultInstanceForType() {
+ return org.apache.drill.exec.proto.BitControl.BitControlHandshake.getDefaultInstance();
+ }
+
+ public org.apache.drill.exec.proto.BitControl.BitControlHandshake build() {
+ org.apache.drill.exec.proto.BitControl.BitControlHandshake result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.BitControlHandshake buildPartial() {
+ org.apache.drill.exec.proto.BitControl.BitControlHandshake result = new org.apache.drill.exec.proto.BitControl.BitControlHandshake(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.rpcVersion_ = rpcVersion_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.channel_ = channel_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ if (endpointBuilder_ == null) {
+ result.endpoint_ = endpoint_;
+ } else {
+ result.endpoint_ = endpointBuilder_.build();
+ }
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.drill.exec.proto.BitControl.BitControlHandshake) {
+ return mergeFrom((org.apache.drill.exec.proto.BitControl.BitControlHandshake)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.drill.exec.proto.BitControl.BitControlHandshake other) {
+ if (other == org.apache.drill.exec.proto.BitControl.BitControlHandshake.getDefaultInstance()) return this;
+ if (other.hasRpcVersion()) {
+ setRpcVersion(other.getRpcVersion());
+ }
+ if (other.hasChannel()) {
+ setChannel(other.getChannel());
+ }
+ if (other.hasEndpoint()) {
+ mergeEndpoint(other.getEndpoint());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.drill.exec.proto.BitControl.BitControlHandshake parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.drill.exec.proto.BitControl.BitControlHandshake) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // optional int32 rpc_version = 1;
+ private int rpcVersion_ ;
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public boolean hasRpcVersion() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public int getRpcVersion() {
+ return rpcVersion_;
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public Builder setRpcVersion(int value) {
+ bitField0_ |= 0x00000001;
+ rpcVersion_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public Builder clearRpcVersion() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ rpcVersion_ = 0;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];
+ private org.apache.drill.exec.proto.UserBitShared.RpcChannel channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.BIT_CONTROL;
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];</code>
+ */
+ public boolean hasChannel() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RpcChannel getChannel() {
+ return channel_;
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];</code>
+ */
+ public Builder setChannel(org.apache.drill.exec.proto.UserBitShared.RpcChannel value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000002;
+ channel_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];</code>
+ */
+ public Builder clearChannel() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.BIT_CONTROL;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.DrillbitEndpoint endpoint = 3;
+ private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> endpointBuilder_;
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public boolean hasEndpoint() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
+ if (endpointBuilder_ == null) {
+ return endpoint_;
+ } else {
+ return endpointBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public Builder setEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+ if (endpointBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ endpoint_ = value;
+ onChanged();
+ } else {
+ endpointBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000004;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public Builder setEndpoint(
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder builderForValue) {
+ if (endpointBuilder_ == null) {
+ endpoint_ = builderForValue.build();
+ onChanged();
+ } else {
+ endpointBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000004;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public Builder mergeEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+ if (endpointBuilder_ == null) {
+ if (((bitField0_ & 0x00000004) == 0x00000004) &&
+ endpoint_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
+ endpoint_ =
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(endpoint_).mergeFrom(value).buildPartial();
+ } else {
+ endpoint_ = value;
+ }
+ onChanged();
+ } else {
+ endpointBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000004;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public Builder clearEndpoint() {
+ if (endpointBuilder_ == null) {
+ endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ onChanged();
+ } else {
+ endpointBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000004);
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getEndpointBuilder() {
+ bitField0_ |= 0x00000004;
+ onChanged();
+ return getEndpointFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
+ if (endpointBuilder_ != null) {
+ return endpointBuilder_.getMessageOrBuilder();
+ } else {
+ return endpoint_;
+ }
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 3;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>
+ getEndpointFieldBuilder() {
+ if (endpointBuilder_ == null) {
+ endpointBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>(
+ endpoint_,
+ getParentForChildren(),
+ isClean());
+ endpoint_ = null;
+ }
+ return endpointBuilder_;
+ }
+
+ // @@protoc_insertion_point(builder_scope:exec.bit.control.BitControlHandshake)
+ }
+
+ static {
+ defaultInstance = new BitControlHandshake(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:exec.bit.control.BitControlHandshake)
+ }
+
+ public interface BitStatusOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // repeated .exec.bit.control.FragmentStatus fragment_status = 1;
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ java.util.List<org.apache.drill.exec.proto.BitControl.FragmentStatus>
+ getFragmentStatusList();
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ org.apache.drill.exec.proto.BitControl.FragmentStatus getFragmentStatus(int index);
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ int getFragmentStatusCount();
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ java.util.List<? extends org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder>
+ getFragmentStatusOrBuilderList();
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder getFragmentStatusOrBuilder(
+ int index);
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.BitStatus}
+ */
+ public static final class BitStatus extends
+ com.google.protobuf.GeneratedMessage
+ implements BitStatusOrBuilder {
+ // Use BitStatus.newBuilder() to construct.
+ private BitStatus(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private BitStatus(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final BitStatus defaultInstance;
+ public static BitStatus getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public BitStatus getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private BitStatus(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 10: {
+ if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+ fragmentStatus_ = new java.util.ArrayList<org.apache.drill.exec.proto.BitControl.FragmentStatus>();
+ mutable_bitField0_ |= 0x00000001;
+ }
+ fragmentStatus_.add(input.readMessage(org.apache.drill.exec.proto.BitControl.FragmentStatus.PARSER, extensionRegistry));
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+ fragmentStatus_ = java.util.Collections.unmodifiableList(fragmentStatus_);
+ }
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitStatus_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitStatus_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.BitStatus.class, org.apache.drill.exec.proto.BitControl.BitStatus.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<BitStatus> PARSER =
+ new com.google.protobuf.AbstractParser<BitStatus>() {
+ public BitStatus parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new BitStatus(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<BitStatus> getParserForType() {
+ return PARSER;
+ }
+
+ // repeated .exec.bit.control.FragmentStatus fragment_status = 1;
+ public static final int FRAGMENT_STATUS_FIELD_NUMBER = 1;
+ private java.util.List<org.apache.drill.exec.proto.BitControl.FragmentStatus> fragmentStatus_;
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public java.util.List<org.apache.drill.exec.proto.BitControl.FragmentStatus> getFragmentStatusList() {
+ return fragmentStatus_;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public java.util.List<? extends org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder>
+ getFragmentStatusOrBuilderList() {
+ return fragmentStatus_;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public int getFragmentStatusCount() {
+ return fragmentStatus_.size();
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus getFragmentStatus(int index) {
+ return fragmentStatus_.get(index);
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder getFragmentStatusOrBuilder(
+ int index) {
+ return fragmentStatus_.get(index);
+ }
+
+ private void initFields() {
+ fragmentStatus_ = java.util.Collections.emptyList();
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ for (int i = 0; i < fragmentStatus_.size(); i++) {
+ output.writeMessage(1, fragmentStatus_.get(i));
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ for (int i = 0; i < fragmentStatus_.size(); i++) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, fragmentStatus_.get(i));
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.BitStatus parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.drill.exec.proto.BitControl.BitStatus prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.BitStatus}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.drill.exec.proto.BitControl.BitStatusOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitStatus_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitStatus_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.BitStatus.class, org.apache.drill.exec.proto.BitControl.BitStatus.Builder.class);
+ }
+
+ // Construct using org.apache.drill.exec.proto.BitControl.BitStatus.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getFragmentStatusFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ if (fragmentStatusBuilder_ == null) {
+ fragmentStatus_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000001);
+ } else {
+ fragmentStatusBuilder_.clear();
+ }
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_BitStatus_descriptor;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.BitStatus getDefaultInstanceForType() {
+ return org.apache.drill.exec.proto.BitControl.BitStatus.getDefaultInstance();
+ }
+
+ public org.apache.drill.exec.proto.BitControl.BitStatus build() {
+ org.apache.drill.exec.proto.BitControl.BitStatus result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.BitStatus buildPartial() {
+ org.apache.drill.exec.proto.BitControl.BitStatus result = new org.apache.drill.exec.proto.BitControl.BitStatus(this);
+ int from_bitField0_ = bitField0_;
+ if (fragmentStatusBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ fragmentStatus_ = java.util.Collections.unmodifiableList(fragmentStatus_);
+ bitField0_ = (bitField0_ & ~0x00000001);
+ }
+ result.fragmentStatus_ = fragmentStatus_;
+ } else {
+ result.fragmentStatus_ = fragmentStatusBuilder_.build();
+ }
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.drill.exec.proto.BitControl.BitStatus) {
+ return mergeFrom((org.apache.drill.exec.proto.BitControl.BitStatus)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.drill.exec.proto.BitControl.BitStatus other) {
+ if (other == org.apache.drill.exec.proto.BitControl.BitStatus.getDefaultInstance()) return this;
+ if (fragmentStatusBuilder_ == null) {
+ if (!other.fragmentStatus_.isEmpty()) {
+ if (fragmentStatus_.isEmpty()) {
+ fragmentStatus_ = other.fragmentStatus_;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ } else {
+ ensureFragmentStatusIsMutable();
+ fragmentStatus_.addAll(other.fragmentStatus_);
+ }
+ onChanged();
+ }
+ } else {
+ if (!other.fragmentStatus_.isEmpty()) {
+ if (fragmentStatusBuilder_.isEmpty()) {
+ fragmentStatusBuilder_.dispose();
+ fragmentStatusBuilder_ = null;
+ fragmentStatus_ = other.fragmentStatus_;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ fragmentStatusBuilder_ =
+ com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+ getFragmentStatusFieldBuilder() : null;
+ } else {
+ fragmentStatusBuilder_.addAllMessages(other.fragmentStatus_);
+ }
+ }
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.drill.exec.proto.BitControl.BitStatus parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.drill.exec.proto.BitControl.BitStatus) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // repeated .exec.bit.control.FragmentStatus fragment_status = 1;
+ private java.util.List<org.apache.drill.exec.proto.BitControl.FragmentStatus> fragmentStatus_ =
+ java.util.Collections.emptyList();
+ private void ensureFragmentStatusIsMutable() {
+ if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+ fragmentStatus_ = new java.util.ArrayList<org.apache.drill.exec.proto.BitControl.FragmentStatus>(fragmentStatus_);
+ bitField0_ |= 0x00000001;
+ }
+ }
+
+ private com.google.protobuf.RepeatedFieldBuilder<
+ org.apache.drill.exec.proto.BitControl.FragmentStatus, org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder, org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder> fragmentStatusBuilder_;
+
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public java.util.List<org.apache.drill.exec.proto.BitControl.FragmentStatus> getFragmentStatusList() {
+ if (fragmentStatusBuilder_ == null) {
+ return java.util.Collections.unmodifiableList(fragmentStatus_);
+ } else {
+ return fragmentStatusBuilder_.getMessageList();
+ }
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public int getFragmentStatusCount() {
+ if (fragmentStatusBuilder_ == null) {
+ return fragmentStatus_.size();
+ } else {
+ return fragmentStatusBuilder_.getCount();
+ }
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus getFragmentStatus(int index) {
+ if (fragmentStatusBuilder_ == null) {
+ return fragmentStatus_.get(index);
+ } else {
+ return fragmentStatusBuilder_.getMessage(index);
+ }
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public Builder setFragmentStatus(
+ int index, org.apache.drill.exec.proto.BitControl.FragmentStatus value) {
+ if (fragmentStatusBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureFragmentStatusIsMutable();
+ fragmentStatus_.set(index, value);
+ onChanged();
+ } else {
+ fragmentStatusBuilder_.setMessage(index, value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public Builder setFragmentStatus(
+ int index, org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder builderForValue) {
+ if (fragmentStatusBuilder_ == null) {
+ ensureFragmentStatusIsMutable();
+ fragmentStatus_.set(index, builderForValue.build());
+ onChanged();
+ } else {
+ fragmentStatusBuilder_.setMessage(index, builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public Builder addFragmentStatus(org.apache.drill.exec.proto.BitControl.FragmentStatus value) {
+ if (fragmentStatusBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureFragmentStatusIsMutable();
+ fragmentStatus_.add(value);
+ onChanged();
+ } else {
+ fragmentStatusBuilder_.addMessage(value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public Builder addFragmentStatus(
+ int index, org.apache.drill.exec.proto.BitControl.FragmentStatus value) {
+ if (fragmentStatusBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureFragmentStatusIsMutable();
+ fragmentStatus_.add(index, value);
+ onChanged();
+ } else {
+ fragmentStatusBuilder_.addMessage(index, value);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public Builder addFragmentStatus(
+ org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder builderForValue) {
+ if (fragmentStatusBuilder_ == null) {
+ ensureFragmentStatusIsMutable();
+ fragmentStatus_.add(builderForValue.build());
+ onChanged();
+ } else {
+ fragmentStatusBuilder_.addMessage(builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public Builder addFragmentStatus(
+ int index, org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder builderForValue) {
+ if (fragmentStatusBuilder_ == null) {
+ ensureFragmentStatusIsMutable();
+ fragmentStatus_.add(index, builderForValue.build());
+ onChanged();
+ } else {
+ fragmentStatusBuilder_.addMessage(index, builderForValue.build());
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public Builder addAllFragmentStatus(
+ java.lang.Iterable<? extends org.apache.drill.exec.proto.BitControl.FragmentStatus> values) {
+ if (fragmentStatusBuilder_ == null) {
+ ensureFragmentStatusIsMutable();
+ super.addAll(values, fragmentStatus_);
+ onChanged();
+ } else {
+ fragmentStatusBuilder_.addAllMessages(values);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public Builder clearFragmentStatus() {
+ if (fragmentStatusBuilder_ == null) {
+ fragmentStatus_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000001);
+ onChanged();
+ } else {
+ fragmentStatusBuilder_.clear();
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public Builder removeFragmentStatus(int index) {
+ if (fragmentStatusBuilder_ == null) {
+ ensureFragmentStatusIsMutable();
+ fragmentStatus_.remove(index);
+ onChanged();
+ } else {
+ fragmentStatusBuilder_.remove(index);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder getFragmentStatusBuilder(
+ int index) {
+ return getFragmentStatusFieldBuilder().getBuilder(index);
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder getFragmentStatusOrBuilder(
+ int index) {
+ if (fragmentStatusBuilder_ == null) {
+ return fragmentStatus_.get(index); } else {
+ return fragmentStatusBuilder_.getMessageOrBuilder(index);
+ }
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public java.util.List<? extends org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder>
+ getFragmentStatusOrBuilderList() {
+ if (fragmentStatusBuilder_ != null) {
+ return fragmentStatusBuilder_.getMessageOrBuilderList();
+ } else {
+ return java.util.Collections.unmodifiableList(fragmentStatus_);
+ }
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder addFragmentStatusBuilder() {
+ return getFragmentStatusFieldBuilder().addBuilder(
+ org.apache.drill.exec.proto.BitControl.FragmentStatus.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder addFragmentStatusBuilder(
+ int index) {
+ return getFragmentStatusFieldBuilder().addBuilder(
+ index, org.apache.drill.exec.proto.BitControl.FragmentStatus.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .exec.bit.control.FragmentStatus fragment_status = 1;</code>
+ */
+ public java.util.List<org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder>
+ getFragmentStatusBuilderList() {
+ return getFragmentStatusFieldBuilder().getBuilderList();
+ }
+ private com.google.protobuf.RepeatedFieldBuilder<
+ org.apache.drill.exec.proto.BitControl.FragmentStatus, org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder, org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder>
+ getFragmentStatusFieldBuilder() {
+ if (fragmentStatusBuilder_ == null) {
+ fragmentStatusBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+ org.apache.drill.exec.proto.BitControl.FragmentStatus, org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder, org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder>(
+ fragmentStatus_,
+ ((bitField0_ & 0x00000001) == 0x00000001),
+ getParentForChildren(),
+ isClean());
+ fragmentStatus_ = null;
+ }
+ return fragmentStatusBuilder_;
+ }
+
+ // @@protoc_insertion_point(builder_scope:exec.bit.control.BitStatus)
+ }
+
+ static {
+ defaultInstance = new BitStatus(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:exec.bit.control.BitStatus)
+ }
+
+ public interface FragmentStatusOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // optional int64 memory_use = 1;
+ /**
+ * <code>optional int64 memory_use = 1;</code>
+ */
+ boolean hasMemoryUse();
+ /**
+ * <code>optional int64 memory_use = 1;</code>
+ */
+ long getMemoryUse();
+
+ // optional int64 batches_completed = 2;
+ /**
+ * <code>optional int64 batches_completed = 2;</code>
+ */
+ boolean hasBatchesCompleted();
+ /**
+ * <code>optional int64 batches_completed = 2;</code>
+ */
+ long getBatchesCompleted();
+
+ // optional int64 records_completed = 3;
+ /**
+ * <code>optional int64 records_completed = 3;</code>
+ */
+ boolean hasRecordsCompleted();
+ /**
+ * <code>optional int64 records_completed = 3;</code>
+ */
+ long getRecordsCompleted();
+
+ // optional int32 estimated_completion_percentage = 4;
+ /**
+ * <code>optional int32 estimated_completion_percentage = 4;</code>
+ */
+ boolean hasEstimatedCompletionPercentage();
+ /**
+ * <code>optional int32 estimated_completion_percentage = 4;</code>
+ */
+ int getEstimatedCompletionPercentage();
+
+ // optional .exec.bit.control.FragmentStatus.FragmentState state = 5;
+ /**
+ * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+ */
+ boolean hasState();
+ /**
+ * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+ */
+ org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState getState();
+
+ // optional int64 data_processed = 6;
+ /**
+ * <code>optional int64 data_processed = 6;</code>
+ */
+ boolean hasDataProcessed();
+ /**
+ * <code>optional int64 data_processed = 6;</code>
+ */
+ long getDataProcessed();
+
+ // optional .exec.bit.FragmentHandle handle = 7;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ boolean hasHandle();
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle();
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder();
+
+ // optional .exec.shared.DrillPBError error = 8;
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ boolean hasError();
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ org.apache.drill.exec.proto.UserBitShared.DrillPBError getError();
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder();
+
+ // optional int64 running_time = 9;
+ /**
+ * <code>optional int64 running_time = 9;</code>
+ */
+ boolean hasRunningTime();
+ /**
+ * <code>optional int64 running_time = 9;</code>
+ */
+ long getRunningTime();
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.FragmentStatus}
+ */
+ public static final class FragmentStatus extends
+ com.google.protobuf.GeneratedMessage
+ implements FragmentStatusOrBuilder {
+ // Use FragmentStatus.newBuilder() to construct.
+ private FragmentStatus(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private FragmentStatus(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final FragmentStatus defaultInstance;
+ public static FragmentStatus getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public FragmentStatus getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private FragmentStatus(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 8: {
+ bitField0_ |= 0x00000001;
+ memoryUse_ = input.readInt64();
+ break;
+ }
+ case 16: {
+ bitField0_ |= 0x00000002;
+ batchesCompleted_ = input.readInt64();
+ break;
+ }
+ case 24: {
+ bitField0_ |= 0x00000004;
+ recordsCompleted_ = input.readInt64();
+ break;
+ }
+ case 32: {
+ bitField0_ |= 0x00000008;
+ estimatedCompletionPercentage_ = input.readInt32();
+ break;
+ }
+ case 40: {
+ int rawValue = input.readEnum();
+ org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState value = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.valueOf(rawValue);
+ if (value == null) {
+ unknownFields.mergeVarintField(5, rawValue);
+ } else {
+ bitField0_ |= 0x00000010;
+ state_ = value;
+ }
+ break;
+ }
+ case 48: {
+ bitField0_ |= 0x00000020;
+ dataProcessed_ = input.readInt64();
+ break;
+ }
+ case 58: {
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ subBuilder = handle_.toBuilder();
+ }
+ handle_ = input.readMessage(org.apache.drill.exec.proto.ExecProtos.FragmentHandle.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(handle_);
+ handle_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000040;
+ break;
+ }
+ case 66: {
+ org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ subBuilder = error_.toBuilder();
+ }
+ error_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.DrillPBError.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(error_);
+ error_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000080;
+ break;
+ }
+ case 72: {
+ bitField0_ |= 0x00000100;
+ runningTime_ = input.readInt64();
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_FragmentStatus_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_FragmentStatus_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.FragmentStatus.class, org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<FragmentStatus> PARSER =
+ new com.google.protobuf.AbstractParser<FragmentStatus>() {
+ public FragmentStatus parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new FragmentStatus(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<FragmentStatus> getParserForType() {
+ return PARSER;
+ }
+
+ /**
+ * Protobuf enum {@code exec.bit.control.FragmentStatus.FragmentState}
+ */
+ public enum FragmentState
+ implements com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * <code>SENDING = 0;</code>
+ */
+ SENDING(0, 0),
+ /**
+ * <code>AWAITING_ALLOCATION = 1;</code>
+ */
+ AWAITING_ALLOCATION(1, 1),
+ /**
+ * <code>RUNNING = 2;</code>
+ */
+ RUNNING(2, 2),
+ /**
+ * <code>FINISHED = 3;</code>
+ */
+ FINISHED(3, 3),
+ /**
+ * <code>CANCELLED = 4;</code>
+ */
+ CANCELLED(4, 4),
+ /**
+ * <code>FAILED = 5;</code>
+ */
+ FAILED(5, 5),
+ ;
+
+ /**
+ * <code>SENDING = 0;</code>
+ */
+ public static final int SENDING_VALUE = 0;
+ /**
+ * <code>AWAITING_ALLOCATION = 1;</code>
+ */
+ public static final int AWAITING_ALLOCATION_VALUE = 1;
+ /**
+ * <code>RUNNING = 2;</code>
+ */
+ public static final int RUNNING_VALUE = 2;
+ /**
+ * <code>FINISHED = 3;</code>
+ */
+ public static final int FINISHED_VALUE = 3;
+ /**
+ * <code>CANCELLED = 4;</code>
+ */
+ public static final int CANCELLED_VALUE = 4;
+ /**
+ * <code>FAILED = 5;</code>
+ */
+ public static final int FAILED_VALUE = 5;
+
+
+ public final int getNumber() { return value; }
+
+ public static FragmentState valueOf(int value) {
+ switch (value) {
+ case 0: return SENDING;
+ case 1: return AWAITING_ALLOCATION;
+ case 2: return RUNNING;
+ case 3: return FINISHED;
+ case 4: return CANCELLED;
+ case 5: return FAILED;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap<FragmentState>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap<FragmentState>
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap<FragmentState>() {
+ public FragmentState findValueByNumber(int number) {
+ return FragmentState.valueOf(number);
+ }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(index);
+ }
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.FragmentStatus.getDescriptor().getEnumTypes().get(0);
+ }
+
+ private static final FragmentState[] VALUES = values();
+
+ public static FragmentState valueOf(
+ com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+
+ private final int index;
+ private final int value;
+
+ private FragmentState(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:exec.bit.control.FragmentStatus.FragmentState)
+ }
+
+ private int bitField0_;
+ // optional int64 memory_use = 1;
+ public static final int MEMORY_USE_FIELD_NUMBER = 1;
+ private long memoryUse_;
+ /**
+ * <code>optional int64 memory_use = 1;</code>
+ */
+ public boolean hasMemoryUse() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional int64 memory_use = 1;</code>
+ */
+ public long getMemoryUse() {
+ return memoryUse_;
+ }
+
+ // optional int64 batches_completed = 2;
+ public static final int BATCHES_COMPLETED_FIELD_NUMBER = 2;
+ private long batchesCompleted_;
+ /**
+ * <code>optional int64 batches_completed = 2;</code>
+ */
+ public boolean hasBatchesCompleted() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional int64 batches_completed = 2;</code>
+ */
+ public long getBatchesCompleted() {
+ return batchesCompleted_;
+ }
+
+ // optional int64 records_completed = 3;
+ public static final int RECORDS_COMPLETED_FIELD_NUMBER = 3;
+ private long recordsCompleted_;
+ /**
+ * <code>optional int64 records_completed = 3;</code>
+ */
+ public boolean hasRecordsCompleted() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional int64 records_completed = 3;</code>
+ */
+ public long getRecordsCompleted() {
+ return recordsCompleted_;
+ }
+
+ // optional int32 estimated_completion_percentage = 4;
+ public static final int ESTIMATED_COMPLETION_PERCENTAGE_FIELD_NUMBER = 4;
+ private int estimatedCompletionPercentage_;
+ /**
+ * <code>optional int32 estimated_completion_percentage = 4;</code>
+ */
+ public boolean hasEstimatedCompletionPercentage() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional int32 estimated_completion_percentage = 4;</code>
+ */
+ public int getEstimatedCompletionPercentage() {
+ return estimatedCompletionPercentage_;
+ }
+
+ // optional .exec.bit.control.FragmentStatus.FragmentState state = 5;
+ public static final int STATE_FIELD_NUMBER = 5;
+ private org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState state_;
+ /**
+ * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+ */
+ public boolean hasState() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+ */
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState getState() {
+ return state_;
+ }
+
+ // optional int64 data_processed = 6;
+ public static final int DATA_PROCESSED_FIELD_NUMBER = 6;
+ private long dataProcessed_;
+ /**
+ * <code>optional int64 data_processed = 6;</code>
+ */
+ public boolean hasDataProcessed() {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+ /**
+ * <code>optional int64 data_processed = 6;</code>
+ */
+ public long getDataProcessed() {
+ return dataProcessed_;
+ }
+
+ // optional .exec.bit.FragmentHandle handle = 7;
+ public static final int HANDLE_FIELD_NUMBER = 7;
+ private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public boolean hasHandle() {
+ return ((bitField0_ & 0x00000040) == 0x00000040);
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
+ return handle_;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
+ return handle_;
+ }
+
+ // optional .exec.shared.DrillPBError error = 8;
+ public static final int ERROR_FIELD_NUMBER = 8;
+ private org.apache.drill.exec.proto.UserBitShared.DrillPBError error_;
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public boolean hasError() {
+ return ((bitField0_ & 0x00000080) == 0x00000080);
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError() {
+ return error_;
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder() {
+ return error_;
+ }
+
+ // optional int64 running_time = 9;
+ public static final int RUNNING_TIME_FIELD_NUMBER = 9;
+ private long runningTime_;
+ /**
+ * <code>optional int64 running_time = 9;</code>
+ */
+ public boolean hasRunningTime() {
+ return ((bitField0_ & 0x00000100) == 0x00000100);
+ }
+ /**
+ * <code>optional int64 running_time = 9;</code>
+ */
+ public long getRunningTime() {
+ return runningTime_;
+ }
+
+ private void initFields() {
+ memoryUse_ = 0L;
+ batchesCompleted_ = 0L;
+ recordsCompleted_ = 0L;
+ estimatedCompletionPercentage_ = 0;
+ state_ = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.SENDING;
+ dataProcessed_ = 0L;
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
+ runningTime_ = 0L;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeInt64(1, memoryUse_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeInt64(2, batchesCompleted_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeInt64(3, recordsCompleted_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeInt32(4, estimatedCompletionPercentage_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ output.writeEnum(5, state_.getNumber());
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ output.writeInt64(6, dataProcessed_);
+ }
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ output.writeMessage(7, handle_);
+ }
+ if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ output.writeMessage(8, error_);
+ }
+ if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ output.writeInt64(9, runningTime_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(1, memoryUse_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(2, batchesCompleted_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(3, recordsCompleted_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt32Size(4, estimatedCompletionPercentage_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(5, state_.getNumber());
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(6, dataProcessed_);
+ }
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(7, handle_);
+ }
+ if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(8, error_);
+ }
+ if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(9, runningTime_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.FragmentStatus parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.drill.exec.proto.BitControl.FragmentStatus prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.FragmentStatus}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.drill.exec.proto.BitControl.FragmentStatusOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_FragmentStatus_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_FragmentStatus_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.FragmentStatus.class, org.apache.drill.exec.proto.BitControl.FragmentStatus.Builder.class);
+ }
+
+ // Construct using org.apache.drill.exec.proto.BitControl.FragmentStatus.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getHandleFieldBuilder();
+ getErrorFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ memoryUse_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ batchesCompleted_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ recordsCompleted_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ estimatedCompletionPercentage_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ state_ = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.SENDING;
+ bitField0_ = (bitField0_ & ~0x00000010);
+ dataProcessed_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000020);
+ if (handleBuilder_ == null) {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ } else {
+ handleBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000040);
+ if (errorBuilder_ == null) {
+ error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
+ } else {
+ errorBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000080);
+ runningTime_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000100);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_FragmentStatus_descriptor;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus getDefaultInstanceForType() {
+ return org.apache.drill.exec.proto.BitControl.FragmentStatus.getDefaultInstance();
+ }
+
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus build() {
+ org.apache.drill.exec.proto.BitControl.FragmentStatus result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus buildPartial() {
+ org.apache.drill.exec.proto.BitControl.FragmentStatus result = new org.apache.drill.exec.proto.BitControl.FragmentStatus(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.memoryUse_ = memoryUse_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.batchesCompleted_ = batchesCompleted_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.recordsCompleted_ = recordsCompleted_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ result.estimatedCompletionPercentage_ = estimatedCompletionPercentage_;
+ if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+ to_bitField0_ |= 0x00000010;
+ }
+ result.state_ = state_;
+ if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+ to_bitField0_ |= 0x00000020;
+ }
+ result.dataProcessed_ = dataProcessed_;
+ if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+ to_bitField0_ |= 0x00000040;
+ }
+ if (handleBuilder_ == null) {
+ result.handle_ = handle_;
+ } else {
+ result.handle_ = handleBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+ to_bitField0_ |= 0x00000080;
+ }
+ if (errorBuilder_ == null) {
+ result.error_ = error_;
+ } else {
+ result.error_ = errorBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+ to_bitField0_ |= 0x00000100;
+ }
+ result.runningTime_ = runningTime_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.drill.exec.proto.BitControl.FragmentStatus) {
+ return mergeFrom((org.apache.drill.exec.proto.BitControl.FragmentStatus)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.drill.exec.proto.BitControl.FragmentStatus other) {
+ if (other == org.apache.drill.exec.proto.BitControl.FragmentStatus.getDefaultInstance()) return this;
+ if (other.hasMemoryUse()) {
+ setMemoryUse(other.getMemoryUse());
+ }
+ if (other.hasBatchesCompleted()) {
+ setBatchesCompleted(other.getBatchesCompleted());
+ }
+ if (other.hasRecordsCompleted()) {
+ setRecordsCompleted(other.getRecordsCompleted());
+ }
+ if (other.hasEstimatedCompletionPercentage()) {
+ setEstimatedCompletionPercentage(other.getEstimatedCompletionPercentage());
+ }
+ if (other.hasState()) {
+ setState(other.getState());
+ }
+ if (other.hasDataProcessed()) {
+ setDataProcessed(other.getDataProcessed());
+ }
+ if (other.hasHandle()) {
+ mergeHandle(other.getHandle());
+ }
+ if (other.hasError()) {
+ mergeError(other.getError());
+ }
+ if (other.hasRunningTime()) {
+ setRunningTime(other.getRunningTime());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.drill.exec.proto.BitControl.FragmentStatus parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.drill.exec.proto.BitControl.FragmentStatus) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // optional int64 memory_use = 1;
+ private long memoryUse_ ;
+ /**
+ * <code>optional int64 memory_use = 1;</code>
+ */
+ public boolean hasMemoryUse() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional int64 memory_use = 1;</code>
+ */
+ public long getMemoryUse() {
+ return memoryUse_;
+ }
+ /**
+ * <code>optional int64 memory_use = 1;</code>
+ */
+ public Builder setMemoryUse(long value) {
+ bitField0_ |= 0x00000001;
+ memoryUse_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int64 memory_use = 1;</code>
+ */
+ public Builder clearMemoryUse() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ memoryUse_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // optional int64 batches_completed = 2;
+ private long batchesCompleted_ ;
+ /**
+ * <code>optional int64 batches_completed = 2;</code>
+ */
+ public boolean hasBatchesCompleted() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional int64 batches_completed = 2;</code>
+ */
+ public long getBatchesCompleted() {
+ return batchesCompleted_;
+ }
+ /**
+ * <code>optional int64 batches_completed = 2;</code>
+ */
+ public Builder setBatchesCompleted(long value) {
+ bitField0_ |= 0x00000002;
+ batchesCompleted_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int64 batches_completed = 2;</code>
+ */
+ public Builder clearBatchesCompleted() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ batchesCompleted_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // optional int64 records_completed = 3;
+ private long recordsCompleted_ ;
+ /**
+ * <code>optional int64 records_completed = 3;</code>
+ */
+ public boolean hasRecordsCompleted() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional int64 records_completed = 3;</code>
+ */
+ public long getRecordsCompleted() {
+ return recordsCompleted_;
+ }
+ /**
+ * <code>optional int64 records_completed = 3;</code>
+ */
+ public Builder setRecordsCompleted(long value) {
+ bitField0_ |= 0x00000004;
+ recordsCompleted_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int64 records_completed = 3;</code>
+ */
+ public Builder clearRecordsCompleted() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ recordsCompleted_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // optional int32 estimated_completion_percentage = 4;
+ private int estimatedCompletionPercentage_ ;
+ /**
+ * <code>optional int32 estimated_completion_percentage = 4;</code>
+ */
+ public boolean hasEstimatedCompletionPercentage() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional int32 estimated_completion_percentage = 4;</code>
+ */
+ public int getEstimatedCompletionPercentage() {
+ return estimatedCompletionPercentage_;
+ }
+ /**
+ * <code>optional int32 estimated_completion_percentage = 4;</code>
+ */
+ public Builder setEstimatedCompletionPercentage(int value) {
+ bitField0_ |= 0x00000008;
+ estimatedCompletionPercentage_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int32 estimated_completion_percentage = 4;</code>
+ */
+ public Builder clearEstimatedCompletionPercentage() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ estimatedCompletionPercentage_ = 0;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.bit.control.FragmentStatus.FragmentState state = 5;
+ private org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState state_ = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.SENDING;
+ /**
+ * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+ */
+ public boolean hasState() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+ */
+ public org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState getState() {
+ return state_;
+ }
+ /**
+ * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+ */
+ public Builder setState(org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000010;
+ state_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+ */
+ public Builder clearState() {
+ bitField0_ = (bitField0_ & ~0x00000010);
+ state_ = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.SENDING;
+ onChanged();
+ return this;
+ }
+
+ // optional int64 data_processed = 6;
+ private long dataProcessed_ ;
+ /**
+ * <code>optional int64 data_processed = 6;</code>
+ */
+ public boolean hasDataProcessed() {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+ /**
+ * <code>optional int64 data_processed = 6;</code>
+ */
+ public long getDataProcessed() {
+ return dataProcessed_;
+ }
+ /**
+ * <code>optional int64 data_processed = 6;</code>
+ */
+ public Builder setDataProcessed(long value) {
+ bitField0_ |= 0x00000020;
+ dataProcessed_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int64 data_processed = 6;</code>
+ */
+ public Builder clearDataProcessed() {
+ bitField0_ = (bitField0_ & ~0x00000020);
+ dataProcessed_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.bit.FragmentHandle handle = 7;
+ private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder> handleBuilder_;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public boolean hasHandle() {
+ return ((bitField0_ & 0x00000040) == 0x00000040);
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
+ if (handleBuilder_ == null) {
+ return handle_;
+ } else {
+ return handleBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public Builder setHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
+ if (handleBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ handle_ = value;
+ onChanged();
+ } else {
+ handleBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000040;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public Builder setHandle(
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder builderForValue) {
+ if (handleBuilder_ == null) {
+ handle_ = builderForValue.build();
+ onChanged();
+ } else {
+ handleBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000040;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public Builder mergeHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
+ if (handleBuilder_ == null) {
+ if (((bitField0_ & 0x00000040) == 0x00000040) &&
+ handle_ != org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance()) {
+ handle_ =
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.newBuilder(handle_).mergeFrom(value).buildPartial();
+ } else {
+ handle_ = value;
+ }
+ onChanged();
+ } else {
+ handleBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000040;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public Builder clearHandle() {
+ if (handleBuilder_ == null) {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ onChanged();
+ } else {
+ handleBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000040);
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder getHandleBuilder() {
+ bitField0_ |= 0x00000040;
+ onChanged();
+ return getHandleFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
+ if (handleBuilder_ != null) {
+ return handleBuilder_.getMessageOrBuilder();
+ } else {
+ return handle_;
+ }
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>
+ getHandleFieldBuilder() {
+ if (handleBuilder_ == null) {
+ handleBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>(
+ handle_,
+ getParentForChildren(),
+ isClean());
+ handle_ = null;
+ }
+ return handleBuilder_;
+ }
+
+ // optional .exec.shared.DrillPBError error = 8;
+ private org.apache.drill.exec.proto.UserBitShared.DrillPBError error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> errorBuilder_;
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public boolean hasError() {
+ return ((bitField0_ & 0x00000080) == 0x00000080);
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError() {
+ if (errorBuilder_ == null) {
+ return error_;
+ } else {
+ return errorBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public Builder setError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
+ if (errorBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ error_ = value;
+ onChanged();
+ } else {
+ errorBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000080;
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public Builder setError(
+ org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
+ if (errorBuilder_ == null) {
+ error_ = builderForValue.build();
+ onChanged();
+ } else {
+ errorBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000080;
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public Builder mergeError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
+ if (errorBuilder_ == null) {
+ if (((bitField0_ & 0x00000080) == 0x00000080) &&
+ error_ != org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance()) {
+ error_ =
+ org.apache.drill.exec.proto.UserBitShared.DrillPBError.newBuilder(error_).mergeFrom(value).buildPartial();
+ } else {
+ error_ = value;
+ }
+ onChanged();
+ } else {
+ errorBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000080;
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public Builder clearError() {
+ if (errorBuilder_ == null) {
+ error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
+ onChanged();
+ } else {
+ errorBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000080);
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder getErrorBuilder() {
+ bitField0_ |= 0x00000080;
+ onChanged();
+ return getErrorFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder() {
+ if (errorBuilder_ != null) {
+ return errorBuilder_.getMessageOrBuilder();
+ } else {
+ return error_;
+ }
+ }
+ /**
+ * <code>optional .exec.shared.DrillPBError error = 8;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder>
+ getErrorFieldBuilder() {
+ if (errorBuilder_ == null) {
+ errorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder>(
+ error_,
+ getParentForChildren(),
+ isClean());
+ error_ = null;
+ }
+ return errorBuilder_;
+ }
+
+ // optional int64 running_time = 9;
+ private long runningTime_ ;
+ /**
+ * <code>optional int64 running_time = 9;</code>
+ */
+ public boolean hasRunningTime() {
+ return ((bitField0_ & 0x00000100) == 0x00000100);
+ }
+ /**
+ * <code>optional int64 running_time = 9;</code>
+ */
+ public long getRunningTime() {
+ return runningTime_;
+ }
+ /**
+ * <code>optional int64 running_time = 9;</code>
+ */
+ public Builder setRunningTime(long value) {
+ bitField0_ |= 0x00000100;
+ runningTime_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int64 running_time = 9;</code>
+ */
+ public Builder clearRunningTime() {
+ bitField0_ = (bitField0_ & ~0x00000100);
+ runningTime_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:exec.bit.control.FragmentStatus)
+ }
+
+ static {
+ defaultInstance = new FragmentStatus(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:exec.bit.control.FragmentStatus)
+ }
+
+ public interface PlanFragmentOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // optional .exec.bit.FragmentHandle handle = 1;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ boolean hasHandle();
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle();
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder();
+
+ // optional float network_cost = 4;
+ /**
+ * <code>optional float network_cost = 4;</code>
+ */
+ boolean hasNetworkCost();
+ /**
+ * <code>optional float network_cost = 4;</code>
+ */
+ float getNetworkCost();
+
+ // optional float cpu_cost = 5;
+ /**
+ * <code>optional float cpu_cost = 5;</code>
+ */
+ boolean hasCpuCost();
+ /**
+ * <code>optional float cpu_cost = 5;</code>
+ */
+ float getCpuCost();
+
+ // optional float disk_cost = 6;
+ /**
+ * <code>optional float disk_cost = 6;</code>
+ */
+ boolean hasDiskCost();
+ /**
+ * <code>optional float disk_cost = 6;</code>
+ */
+ float getDiskCost();
+
+ // optional float memory_cost = 7;
+ /**
+ * <code>optional float memory_cost = 7;</code>
+ */
+ boolean hasMemoryCost();
+ /**
+ * <code>optional float memory_cost = 7;</code>
+ */
+ float getMemoryCost();
+
+ // optional string fragment_json = 8;
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ boolean hasFragmentJson();
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ java.lang.String getFragmentJson();
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ com.google.protobuf.ByteString
+ getFragmentJsonBytes();
+
+ // optional .exec.DrillbitEndpoint assignment = 10;
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ boolean hasAssignment();
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getAssignment();
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getAssignmentOrBuilder();
+
+ // optional bool leaf_fragment = 9;
+ /**
+ * <code>optional bool leaf_fragment = 9;</code>
+ */
+ boolean hasLeafFragment();
+ /**
+ * <code>optional bool leaf_fragment = 9;</code>
+ */
+ boolean getLeafFragment();
+
+ // optional .exec.DrillbitEndpoint foreman = 11;
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ boolean hasForeman();
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getForeman();
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getForemanOrBuilder();
+
+ // optional int64 mem_initial = 12 [default = 20000000];
+ /**
+ * <code>optional int64 mem_initial = 12 [default = 20000000];</code>
+ *
+ * <pre>
+ * 20 megs
+ * </pre>
+ */
+ boolean hasMemInitial();
+ /**
+ * <code>optional int64 mem_initial = 12 [default = 20000000];</code>
+ *
+ * <pre>
+ * 20 megs
+ * </pre>
+ */
+ long getMemInitial();
+
+ // optional int64 mem_max = 13 [default = 20000000000];
+ /**
+ * <code>optional int64 mem_max = 13 [default = 20000000000];</code>
+ *
+ * <pre>
+ * 20 gigs
+ * </pre>
+ */
+ boolean hasMemMax();
+ /**
+ * <code>optional int64 mem_max = 13 [default = 20000000000];</code>
+ *
+ * <pre>
+ * 20 gigs
+ * </pre>
+ */
+ long getMemMax();
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.PlanFragment}
+ */
+ public static final class PlanFragment extends
+ com.google.protobuf.GeneratedMessage
+ implements PlanFragmentOrBuilder {
+ // Use PlanFragment.newBuilder() to construct.
+ private PlanFragment(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private PlanFragment(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final PlanFragment defaultInstance;
+ public static PlanFragment getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public PlanFragment getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private PlanFragment(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 10: {
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ subBuilder = handle_.toBuilder();
+ }
+ handle_ = input.readMessage(org.apache.drill.exec.proto.ExecProtos.FragmentHandle.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(handle_);
+ handle_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000001;
+ break;
+ }
+ case 37: {
+ bitField0_ |= 0x00000002;
+ networkCost_ = input.readFloat();
+ break;
+ }
+ case 45: {
+ bitField0_ |= 0x00000004;
+ cpuCost_ = input.readFloat();
+ break;
+ }
+ case 53: {
+ bitField0_ |= 0x00000008;
+ diskCost_ = input.readFloat();
+ break;
+ }
+ case 61: {
+ bitField0_ |= 0x00000010;
+ memoryCost_ = input.readFloat();
+ break;
+ }
+ case 66: {
+ bitField0_ |= 0x00000020;
+ fragmentJson_ = input.readBytes();
+ break;
+ }
+ case 72: {
+ bitField0_ |= 0x00000080;
+ leafFragment_ = input.readBool();
+ break;
+ }
+ case 82: {
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ subBuilder = assignment_.toBuilder();
+ }
+ assignment_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(assignment_);
+ assignment_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000040;
+ break;
+ }
+ case 90: {
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ subBuilder = foreman_.toBuilder();
+ }
+ foreman_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(foreman_);
+ foreman_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000100;
+ break;
+ }
+ case 96: {
+ bitField0_ |= 0x00000200;
+ memInitial_ = input.readInt64();
+ break;
+ }
+ case 104: {
+ bitField0_ |= 0x00000400;
+ memMax_ = input.readInt64();
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_PlanFragment_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_PlanFragment_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.PlanFragment.class, org.apache.drill.exec.proto.BitControl.PlanFragment.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<PlanFragment> PARSER =
+ new com.google.protobuf.AbstractParser<PlanFragment>() {
+ public PlanFragment parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new PlanFragment(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<PlanFragment> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // optional .exec.bit.FragmentHandle handle = 1;
+ public static final int HANDLE_FIELD_NUMBER = 1;
+ private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public boolean hasHandle() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
+ return handle_;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
+ return handle_;
+ }
+
+ // optional float network_cost = 4;
+ public static final int NETWORK_COST_FIELD_NUMBER = 4;
+ private float networkCost_;
+ /**
+ * <code>optional float network_cost = 4;</code>
+ */
+ public boolean hasNetworkCost() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional float network_cost = 4;</code>
+ */
+ public float getNetworkCost() {
+ return networkCost_;
+ }
+
+ // optional float cpu_cost = 5;
+ public static final int CPU_COST_FIELD_NUMBER = 5;
+ private float cpuCost_;
+ /**
+ * <code>optional float cpu_cost = 5;</code>
+ */
+ public boolean hasCpuCost() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional float cpu_cost = 5;</code>
+ */
+ public float getCpuCost() {
+ return cpuCost_;
+ }
+
+ // optional float disk_cost = 6;
+ public static final int DISK_COST_FIELD_NUMBER = 6;
+ private float diskCost_;
+ /**
+ * <code>optional float disk_cost = 6;</code>
+ */
+ public boolean hasDiskCost() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional float disk_cost = 6;</code>
+ */
+ public float getDiskCost() {
+ return diskCost_;
+ }
+
+ // optional float memory_cost = 7;
+ public static final int MEMORY_COST_FIELD_NUMBER = 7;
+ private float memoryCost_;
+ /**
+ * <code>optional float memory_cost = 7;</code>
+ */
+ public boolean hasMemoryCost() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * <code>optional float memory_cost = 7;</code>
+ */
+ public float getMemoryCost() {
+ return memoryCost_;
+ }
+
+ // optional string fragment_json = 8;
+ public static final int FRAGMENT_JSON_FIELD_NUMBER = 8;
+ private java.lang.Object fragmentJson_;
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ public boolean hasFragmentJson() {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ public java.lang.String getFragmentJson() {
+ java.lang.Object ref = fragmentJson_;
+ if (ref instanceof java.lang.String) {
+ return (java.lang.String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ java.lang.String s = bs.toStringUtf8();
+ if (bs.isValidUtf8()) {
+ fragmentJson_ = s;
+ }
+ return s;
+ }
+ }
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ public com.google.protobuf.ByteString
+ getFragmentJsonBytes() {
+ java.lang.Object ref = fragmentJson_;
+ if (ref instanceof java.lang.String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ fragmentJson_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ // optional .exec.DrillbitEndpoint assignment = 10;
+ public static final int ASSIGNMENT_FIELD_NUMBER = 10;
+ private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint assignment_;
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public boolean hasAssignment() {
+ return ((bitField0_ & 0x00000040) == 0x00000040);
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getAssignment() {
+ return assignment_;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getAssignmentOrBuilder() {
+ return assignment_;
+ }
+
+ // optional bool leaf_fragment = 9;
+ public static final int LEAF_FRAGMENT_FIELD_NUMBER = 9;
+ private boolean leafFragment_;
+ /**
+ * <code>optional bool leaf_fragment = 9;</code>
+ */
+ public boolean hasLeafFragment() {
+ return ((bitField0_ & 0x00000080) == 0x00000080);
+ }
+ /**
+ * <code>optional bool leaf_fragment = 9;</code>
+ */
+ public boolean getLeafFragment() {
+ return leafFragment_;
+ }
+
+ // optional .exec.DrillbitEndpoint foreman = 11;
+ public static final int FOREMAN_FIELD_NUMBER = 11;
+ private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint foreman_;
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public boolean hasForeman() {
+ return ((bitField0_ & 0x00000100) == 0x00000100);
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getForeman() {
+ return foreman_;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getForemanOrBuilder() {
+ return foreman_;
+ }
+
+ // optional int64 mem_initial = 12 [default = 20000000];
+ public static final int MEM_INITIAL_FIELD_NUMBER = 12;
+ private long memInitial_;
+ /**
+ * <code>optional int64 mem_initial = 12 [default = 20000000];</code>
+ *
+ * <pre>
+ * 20 megs
+ * </pre>
+ */
+ public boolean hasMemInitial() {
+ return ((bitField0_ & 0x00000200) == 0x00000200);
+ }
+ /**
+ * <code>optional int64 mem_initial = 12 [default = 20000000];</code>
+ *
+ * <pre>
+ * 20 megs
+ * </pre>
+ */
+ public long getMemInitial() {
+ return memInitial_;
+ }
+
+ // optional int64 mem_max = 13 [default = 20000000000];
+ public static final int MEM_MAX_FIELD_NUMBER = 13;
+ private long memMax_;
+ /**
+ * <code>optional int64 mem_max = 13 [default = 20000000000];</code>
+ *
+ * <pre>
+ * 20 gigs
+ * </pre>
+ */
+ public boolean hasMemMax() {
+ return ((bitField0_ & 0x00000400) == 0x00000400);
+ }
+ /**
+ * <code>optional int64 mem_max = 13 [default = 20000000000];</code>
+ *
+ * <pre>
+ * 20 gigs
+ * </pre>
+ */
+ public long getMemMax() {
+ return memMax_;
+ }
+
+ private void initFields() {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ networkCost_ = 0F;
+ cpuCost_ = 0F;
+ diskCost_ = 0F;
+ memoryCost_ = 0F;
+ fragmentJson_ = "";
+ assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ leafFragment_ = false;
+ foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ memInitial_ = 20000000L;
+ memMax_ = 20000000000L;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeMessage(1, handle_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeFloat(4, networkCost_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeFloat(5, cpuCost_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeFloat(6, diskCost_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ output.writeFloat(7, memoryCost_);
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ output.writeBytes(8, getFragmentJsonBytes());
+ }
+ if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ output.writeBool(9, leafFragment_);
+ }
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ output.writeMessage(10, assignment_);
+ }
+ if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ output.writeMessage(11, foreman_);
+ }
+ if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ output.writeInt64(12, memInitial_);
+ }
+ if (((bitField0_ & 0x00000400) == 0x00000400)) {
+ output.writeInt64(13, memMax_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, handle_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeFloatSize(4, networkCost_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeFloatSize(5, cpuCost_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeFloatSize(6, diskCost_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeFloatSize(7, memoryCost_);
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(8, getFragmentJsonBytes());
+ }
+ if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(9, leafFragment_);
+ }
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(10, assignment_);
+ }
+ if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(11, foreman_);
+ }
+ if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(12, memInitial_);
+ }
+ if (((bitField0_ & 0x00000400) == 0x00000400)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(13, memMax_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.PlanFragment parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.drill.exec.proto.BitControl.PlanFragment prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.PlanFragment}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.drill.exec.proto.BitControl.PlanFragmentOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_PlanFragment_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_PlanFragment_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.PlanFragment.class, org.apache.drill.exec.proto.BitControl.PlanFragment.Builder.class);
+ }
+
+ // Construct using org.apache.drill.exec.proto.BitControl.PlanFragment.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getHandleFieldBuilder();
+ getAssignmentFieldBuilder();
+ getForemanFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ if (handleBuilder_ == null) {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ } else {
+ handleBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ networkCost_ = 0F;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ cpuCost_ = 0F;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ diskCost_ = 0F;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ memoryCost_ = 0F;
+ bitField0_ = (bitField0_ & ~0x00000010);
+ fragmentJson_ = "";
+ bitField0_ = (bitField0_ & ~0x00000020);
+ if (assignmentBuilder_ == null) {
+ assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ } else {
+ assignmentBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000040);
+ leafFragment_ = false;
+ bitField0_ = (bitField0_ & ~0x00000080);
+ if (foremanBuilder_ == null) {
+ foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ } else {
+ foremanBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000100);
+ memInitial_ = 20000000L;
+ bitField0_ = (bitField0_ & ~0x00000200);
+ memMax_ = 20000000000L;
+ bitField0_ = (bitField0_ & ~0x00000400);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_PlanFragment_descriptor;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.PlanFragment getDefaultInstanceForType() {
+ return org.apache.drill.exec.proto.BitControl.PlanFragment.getDefaultInstance();
+ }
+
+ public org.apache.drill.exec.proto.BitControl.PlanFragment build() {
+ org.apache.drill.exec.proto.BitControl.PlanFragment result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.PlanFragment buildPartial() {
+ org.apache.drill.exec.proto.BitControl.PlanFragment result = new org.apache.drill.exec.proto.BitControl.PlanFragment(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ if (handleBuilder_ == null) {
+ result.handle_ = handle_;
+ } else {
+ result.handle_ = handleBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.networkCost_ = networkCost_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.cpuCost_ = cpuCost_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ result.diskCost_ = diskCost_;
+ if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+ to_bitField0_ |= 0x00000010;
+ }
+ result.memoryCost_ = memoryCost_;
+ if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+ to_bitField0_ |= 0x00000020;
+ }
+ result.fragmentJson_ = fragmentJson_;
+ if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+ to_bitField0_ |= 0x00000040;
+ }
+ if (assignmentBuilder_ == null) {
+ result.assignment_ = assignment_;
+ } else {
+ result.assignment_ = assignmentBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+ to_bitField0_ |= 0x00000080;
+ }
+ result.leafFragment_ = leafFragment_;
+ if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+ to_bitField0_ |= 0x00000100;
+ }
+ if (foremanBuilder_ == null) {
+ result.foreman_ = foreman_;
+ } else {
+ result.foreman_ = foremanBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
+ to_bitField0_ |= 0x00000200;
+ }
+ result.memInitial_ = memInitial_;
+ if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
+ to_bitField0_ |= 0x00000400;
+ }
+ result.memMax_ = memMax_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.drill.exec.proto.BitControl.PlanFragment) {
+ return mergeFrom((org.apache.drill.exec.proto.BitControl.PlanFragment)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.drill.exec.proto.BitControl.PlanFragment other) {
+ if (other == org.apache.drill.exec.proto.BitControl.PlanFragment.getDefaultInstance()) return this;
+ if (other.hasHandle()) {
+ mergeHandle(other.getHandle());
+ }
+ if (other.hasNetworkCost()) {
+ setNetworkCost(other.getNetworkCost());
+ }
+ if (other.hasCpuCost()) {
+ setCpuCost(other.getCpuCost());
+ }
+ if (other.hasDiskCost()) {
+ setDiskCost(other.getDiskCost());
+ }
+ if (other.hasMemoryCost()) {
+ setMemoryCost(other.getMemoryCost());
+ }
+ if (other.hasFragmentJson()) {
+ bitField0_ |= 0x00000020;
+ fragmentJson_ = other.fragmentJson_;
+ onChanged();
+ }
+ if (other.hasAssignment()) {
+ mergeAssignment(other.getAssignment());
+ }
+ if (other.hasLeafFragment()) {
+ setLeafFragment(other.getLeafFragment());
+ }
+ if (other.hasForeman()) {
+ mergeForeman(other.getForeman());
+ }
+ if (other.hasMemInitial()) {
+ setMemInitial(other.getMemInitial());
+ }
+ if (other.hasMemMax()) {
+ setMemMax(other.getMemMax());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.drill.exec.proto.BitControl.PlanFragment parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.drill.exec.proto.BitControl.PlanFragment) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // optional .exec.bit.FragmentHandle handle = 1;
+ private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder> handleBuilder_;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public boolean hasHandle() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
+ if (handleBuilder_ == null) {
+ return handle_;
+ } else {
+ return handleBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public Builder setHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
+ if (handleBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ handle_ = value;
+ onChanged();
+ } else {
+ handleBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public Builder setHandle(
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder builderForValue) {
+ if (handleBuilder_ == null) {
+ handle_ = builderForValue.build();
+ onChanged();
+ } else {
+ handleBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public Builder mergeHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
+ if (handleBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001) &&
+ handle_ != org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance()) {
+ handle_ =
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.newBuilder(handle_).mergeFrom(value).buildPartial();
+ } else {
+ handle_ = value;
+ }
+ onChanged();
+ } else {
+ handleBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public Builder clearHandle() {
+ if (handleBuilder_ == null) {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ onChanged();
+ } else {
+ handleBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder getHandleBuilder() {
+ bitField0_ |= 0x00000001;
+ onChanged();
+ return getHandleFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
+ if (handleBuilder_ != null) {
+ return handleBuilder_.getMessageOrBuilder();
+ } else {
+ return handle_;
+ }
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>
+ getHandleFieldBuilder() {
+ if (handleBuilder_ == null) {
+ handleBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>(
+ handle_,
+ getParentForChildren(),
+ isClean());
+ handle_ = null;
+ }
+ return handleBuilder_;
+ }
+
+ // optional float network_cost = 4;
+ private float networkCost_ ;
+ /**
+ * <code>optional float network_cost = 4;</code>
+ */
+ public boolean hasNetworkCost() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional float network_cost = 4;</code>
+ */
+ public float getNetworkCost() {
+ return networkCost_;
+ }
+ /**
+ * <code>optional float network_cost = 4;</code>
+ */
+ public Builder setNetworkCost(float value) {
+ bitField0_ |= 0x00000002;
+ networkCost_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional float network_cost = 4;</code>
+ */
+ public Builder clearNetworkCost() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ networkCost_ = 0F;
+ onChanged();
+ return this;
+ }
+
+ // optional float cpu_cost = 5;
+ private float cpuCost_ ;
+ /**
+ * <code>optional float cpu_cost = 5;</code>
+ */
+ public boolean hasCpuCost() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional float cpu_cost = 5;</code>
+ */
+ public float getCpuCost() {
+ return cpuCost_;
+ }
+ /**
+ * <code>optional float cpu_cost = 5;</code>
+ */
+ public Builder setCpuCost(float value) {
+ bitField0_ |= 0x00000004;
+ cpuCost_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional float cpu_cost = 5;</code>
+ */
+ public Builder clearCpuCost() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ cpuCost_ = 0F;
+ onChanged();
+ return this;
+ }
+
+ // optional float disk_cost = 6;
+ private float diskCost_ ;
+ /**
+ * <code>optional float disk_cost = 6;</code>
+ */
+ public boolean hasDiskCost() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional float disk_cost = 6;</code>
+ */
+ public float getDiskCost() {
+ return diskCost_;
+ }
+ /**
+ * <code>optional float disk_cost = 6;</code>
+ */
+ public Builder setDiskCost(float value) {
+ bitField0_ |= 0x00000008;
+ diskCost_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional float disk_cost = 6;</code>
+ */
+ public Builder clearDiskCost() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ diskCost_ = 0F;
+ onChanged();
+ return this;
+ }
+
+ // optional float memory_cost = 7;
+ private float memoryCost_ ;
+ /**
+ * <code>optional float memory_cost = 7;</code>
+ */
+ public boolean hasMemoryCost() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * <code>optional float memory_cost = 7;</code>
+ */
+ public float getMemoryCost() {
+ return memoryCost_;
+ }
+ /**
+ * <code>optional float memory_cost = 7;</code>
+ */
+ public Builder setMemoryCost(float value) {
+ bitField0_ |= 0x00000010;
+ memoryCost_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional float memory_cost = 7;</code>
+ */
+ public Builder clearMemoryCost() {
+ bitField0_ = (bitField0_ & ~0x00000010);
+ memoryCost_ = 0F;
+ onChanged();
+ return this;
+ }
+
+ // optional string fragment_json = 8;
+ private java.lang.Object fragmentJson_ = "";
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ public boolean hasFragmentJson() {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ public java.lang.String getFragmentJson() {
+ java.lang.Object ref = fragmentJson_;
+ if (!(ref instanceof java.lang.String)) {
+ java.lang.String s = ((com.google.protobuf.ByteString) ref)
+ .toStringUtf8();
+ fragmentJson_ = s;
+ return s;
+ } else {
+ return (java.lang.String) ref;
+ }
+ }
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ public com.google.protobuf.ByteString
+ getFragmentJsonBytes() {
+ java.lang.Object ref = fragmentJson_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ fragmentJson_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ public Builder setFragmentJson(
+ java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000020;
+ fragmentJson_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ public Builder clearFragmentJson() {
+ bitField0_ = (bitField0_ & ~0x00000020);
+ fragmentJson_ = getDefaultInstance().getFragmentJson();
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional string fragment_json = 8;</code>
+ */
+ public Builder setFragmentJsonBytes(
+ com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000020;
+ fragmentJson_ = value;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.DrillbitEndpoint assignment = 10;
+ private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> assignmentBuilder_;
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public boolean hasAssignment() {
+ return ((bitField0_ & 0x00000040) == 0x00000040);
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getAssignment() {
+ if (assignmentBuilder_ == null) {
+ return assignment_;
+ } else {
+ return assignmentBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public Builder setAssignment(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+ if (assignmentBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ assignment_ = value;
+ onChanged();
+ } else {
+ assignmentBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000040;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public Builder setAssignment(
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder builderForValue) {
+ if (assignmentBuilder_ == null) {
+ assignment_ = builderForValue.build();
+ onChanged();
+ } else {
+ assignmentBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000040;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public Builder mergeAssignment(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+ if (assignmentBuilder_ == null) {
+ if (((bitField0_ & 0x00000040) == 0x00000040) &&
+ assignment_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
+ assignment_ =
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(assignment_).mergeFrom(value).buildPartial();
+ } else {
+ assignment_ = value;
+ }
+ onChanged();
+ } else {
+ assignmentBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000040;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public Builder clearAssignment() {
+ if (assignmentBuilder_ == null) {
+ assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ onChanged();
+ } else {
+ assignmentBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000040);
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getAssignmentBuilder() {
+ bitField0_ |= 0x00000040;
+ onChanged();
+ return getAssignmentFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getAssignmentOrBuilder() {
+ if (assignmentBuilder_ != null) {
+ return assignmentBuilder_.getMessageOrBuilder();
+ } else {
+ return assignment_;
+ }
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>
+ getAssignmentFieldBuilder() {
+ if (assignmentBuilder_ == null) {
+ assignmentBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>(
+ assignment_,
+ getParentForChildren(),
+ isClean());
+ assignment_ = null;
+ }
+ return assignmentBuilder_;
+ }
+
+ // optional bool leaf_fragment = 9;
+ private boolean leafFragment_ ;
+ /**
+ * <code>optional bool leaf_fragment = 9;</code>
+ */
+ public boolean hasLeafFragment() {
+ return ((bitField0_ & 0x00000080) == 0x00000080);
+ }
+ /**
+ * <code>optional bool leaf_fragment = 9;</code>
+ */
+ public boolean getLeafFragment() {
+ return leafFragment_;
+ }
+ /**
+ * <code>optional bool leaf_fragment = 9;</code>
+ */
+ public Builder setLeafFragment(boolean value) {
+ bitField0_ |= 0x00000080;
+ leafFragment_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bool leaf_fragment = 9;</code>
+ */
+ public Builder clearLeafFragment() {
+ bitField0_ = (bitField0_ & ~0x00000080);
+ leafFragment_ = false;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.DrillbitEndpoint foreman = 11;
+ private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> foremanBuilder_;
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public boolean hasForeman() {
+ return ((bitField0_ & 0x00000100) == 0x00000100);
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getForeman() {
+ if (foremanBuilder_ == null) {
+ return foreman_;
+ } else {
+ return foremanBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public Builder setForeman(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+ if (foremanBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ foreman_ = value;
+ onChanged();
+ } else {
+ foremanBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000100;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public Builder setForeman(
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder builderForValue) {
+ if (foremanBuilder_ == null) {
+ foreman_ = builderForValue.build();
+ onChanged();
+ } else {
+ foremanBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000100;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public Builder mergeForeman(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+ if (foremanBuilder_ == null) {
+ if (((bitField0_ & 0x00000100) == 0x00000100) &&
+ foreman_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
+ foreman_ =
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(foreman_).mergeFrom(value).buildPartial();
+ } else {
+ foreman_ = value;
+ }
+ onChanged();
+ } else {
+ foremanBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000100;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public Builder clearForeman() {
+ if (foremanBuilder_ == null) {
+ foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ onChanged();
+ } else {
+ foremanBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000100);
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getForemanBuilder() {
+ bitField0_ |= 0x00000100;
+ onChanged();
+ return getForemanFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getForemanOrBuilder() {
+ if (foremanBuilder_ != null) {
+ return foremanBuilder_.getMessageOrBuilder();
+ } else {
+ return foreman_;
+ }
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>
+ getForemanFieldBuilder() {
+ if (foremanBuilder_ == null) {
+ foremanBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>(
+ foreman_,
+ getParentForChildren(),
+ isClean());
+ foreman_ = null;
+ }
+ return foremanBuilder_;
+ }
+
+ // optional int64 mem_initial = 12 [default = 20000000];
+ private long memInitial_ = 20000000L;
+ /**
+ * <code>optional int64 mem_initial = 12 [default = 20000000];</code>
+ *
+ * <pre>
+ * 20 megs
+ * </pre>
+ */
+ public boolean hasMemInitial() {
+ return ((bitField0_ & 0x00000200) == 0x00000200);
+ }
+ /**
+ * <code>optional int64 mem_initial = 12 [default = 20000000];</code>
+ *
+ * <pre>
+ * 20 megs
+ * </pre>
+ */
+ public long getMemInitial() {
+ return memInitial_;
+ }
+ /**
+ * <code>optional int64 mem_initial = 12 [default = 20000000];</code>
+ *
+ * <pre>
+ * 20 megs
+ * </pre>
+ */
+ public Builder setMemInitial(long value) {
+ bitField0_ |= 0x00000200;
+ memInitial_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int64 mem_initial = 12 [default = 20000000];</code>
+ *
+ * <pre>
+ * 20 megs
+ * </pre>
+ */
+ public Builder clearMemInitial() {
+ bitField0_ = (bitField0_ & ~0x00000200);
+ memInitial_ = 20000000L;
+ onChanged();
+ return this;
+ }
+
+ // optional int64 mem_max = 13 [default = 20000000000];
+ private long memMax_ = 20000000000L;
+ /**
+ * <code>optional int64 mem_max = 13 [default = 20000000000];</code>
+ *
+ * <pre>
+ * 20 gigs
+ * </pre>
+ */
+ public boolean hasMemMax() {
+ return ((bitField0_ & 0x00000400) == 0x00000400);
+ }
+ /**
+ * <code>optional int64 mem_max = 13 [default = 20000000000];</code>
+ *
+ * <pre>
+ * 20 gigs
+ * </pre>
+ */
+ public long getMemMax() {
+ return memMax_;
+ }
+ /**
+ * <code>optional int64 mem_max = 13 [default = 20000000000];</code>
+ *
+ * <pre>
+ * 20 gigs
+ * </pre>
+ */
+ public Builder setMemMax(long value) {
+ bitField0_ |= 0x00000400;
+ memMax_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int64 mem_max = 13 [default = 20000000000];</code>
+ *
+ * <pre>
+ * 20 gigs
+ * </pre>
+ */
+ public Builder clearMemMax() {
+ bitField0_ = (bitField0_ & ~0x00000400);
+ memMax_ = 20000000000L;
+ onChanged();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:exec.bit.control.PlanFragment)
+ }
+
+ static {
+ defaultInstance = new PlanFragment(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:exec.bit.control.PlanFragment)
+ }
+
+ public interface WorkQueueStatusOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // optional .exec.DrillbitEndpoint endpoint = 1;
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ boolean hasEndpoint();
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint();
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder();
+
+ // optional int32 queue_length = 2;
+ /**
+ * <code>optional int32 queue_length = 2;</code>
+ */
+ boolean hasQueueLength();
+ /**
+ * <code>optional int32 queue_length = 2;</code>
+ */
+ int getQueueLength();
+
+ // optional int64 report_time = 3;
+ /**
+ * <code>optional int64 report_time = 3;</code>
+ */
+ boolean hasReportTime();
+ /**
+ * <code>optional int64 report_time = 3;</code>
+ */
+ long getReportTime();
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.WorkQueueStatus}
+ */
+ public static final class WorkQueueStatus extends
+ com.google.protobuf.GeneratedMessage
+ implements WorkQueueStatusOrBuilder {
+ // Use WorkQueueStatus.newBuilder() to construct.
+ private WorkQueueStatus(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private WorkQueueStatus(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final WorkQueueStatus defaultInstance;
+ public static WorkQueueStatus getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public WorkQueueStatus getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private WorkQueueStatus(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 10: {
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ subBuilder = endpoint_.toBuilder();
+ }
+ endpoint_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(endpoint_);
+ endpoint_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000001;
+ break;
+ }
+ case 16: {
+ bitField0_ |= 0x00000002;
+ queueLength_ = input.readInt32();
+ break;
+ }
+ case 24: {
+ bitField0_ |= 0x00000004;
+ reportTime_ = input.readInt64();
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_WorkQueueStatus_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_WorkQueueStatus_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.WorkQueueStatus.class, org.apache.drill.exec.proto.BitControl.WorkQueueStatus.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<WorkQueueStatus> PARSER =
+ new com.google.protobuf.AbstractParser<WorkQueueStatus>() {
+ public WorkQueueStatus parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new WorkQueueStatus(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<WorkQueueStatus> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // optional .exec.DrillbitEndpoint endpoint = 1;
+ public static final int ENDPOINT_FIELD_NUMBER = 1;
+ private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_;
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public boolean hasEndpoint() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
+ return endpoint_;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
+ return endpoint_;
+ }
+
+ // optional int32 queue_length = 2;
+ public static final int QUEUE_LENGTH_FIELD_NUMBER = 2;
+ private int queueLength_;
+ /**
+ * <code>optional int32 queue_length = 2;</code>
+ */
+ public boolean hasQueueLength() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional int32 queue_length = 2;</code>
+ */
+ public int getQueueLength() {
+ return queueLength_;
+ }
+
+ // optional int64 report_time = 3;
+ public static final int REPORT_TIME_FIELD_NUMBER = 3;
+ private long reportTime_;
+ /**
+ * <code>optional int64 report_time = 3;</code>
+ */
+ public boolean hasReportTime() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional int64 report_time = 3;</code>
+ */
+ public long getReportTime() {
+ return reportTime_;
+ }
+
+ private void initFields() {
+ endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ queueLength_ = 0;
+ reportTime_ = 0L;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeMessage(1, endpoint_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeInt32(2, queueLength_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeInt64(3, reportTime_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, endpoint_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt32Size(2, queueLength_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(3, reportTime_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitControl.WorkQueueStatus parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.drill.exec.proto.BitControl.WorkQueueStatus prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code exec.bit.control.WorkQueueStatus}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.drill.exec.proto.BitControl.WorkQueueStatusOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_WorkQueueStatus_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_WorkQueueStatus_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitControl.WorkQueueStatus.class, org.apache.drill.exec.proto.BitControl.WorkQueueStatus.Builder.class);
+ }
+
+ // Construct using org.apache.drill.exec.proto.BitControl.WorkQueueStatus.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getEndpointFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ if (endpointBuilder_ == null) {
+ endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ } else {
+ endpointBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ queueLength_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ reportTime_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_WorkQueueStatus_descriptor;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.WorkQueueStatus getDefaultInstanceForType() {
+ return org.apache.drill.exec.proto.BitControl.WorkQueueStatus.getDefaultInstance();
+ }
+
+ public org.apache.drill.exec.proto.BitControl.WorkQueueStatus build() {
+ org.apache.drill.exec.proto.BitControl.WorkQueueStatus result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.drill.exec.proto.BitControl.WorkQueueStatus buildPartial() {
+ org.apache.drill.exec.proto.BitControl.WorkQueueStatus result = new org.apache.drill.exec.proto.BitControl.WorkQueueStatus(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ if (endpointBuilder_ == null) {
+ result.endpoint_ = endpoint_;
+ } else {
+ result.endpoint_ = endpointBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.queueLength_ = queueLength_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.reportTime_ = reportTime_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.drill.exec.proto.BitControl.WorkQueueStatus) {
+ return mergeFrom((org.apache.drill.exec.proto.BitControl.WorkQueueStatus)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.drill.exec.proto.BitControl.WorkQueueStatus other) {
+ if (other == org.apache.drill.exec.proto.BitControl.WorkQueueStatus.getDefaultInstance()) return this;
+ if (other.hasEndpoint()) {
+ mergeEndpoint(other.getEndpoint());
+ }
+ if (other.hasQueueLength()) {
+ setQueueLength(other.getQueueLength());
+ }
+ if (other.hasReportTime()) {
+ setReportTime(other.getReportTime());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.drill.exec.proto.BitControl.WorkQueueStatus parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.drill.exec.proto.BitControl.WorkQueueStatus) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // optional .exec.DrillbitEndpoint endpoint = 1;
+ private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> endpointBuilder_;
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public boolean hasEndpoint() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
+ if (endpointBuilder_ == null) {
+ return endpoint_;
+ } else {
+ return endpointBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public Builder setEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+ if (endpointBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ endpoint_ = value;
+ onChanged();
+ } else {
+ endpointBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public Builder setEndpoint(
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder builderForValue) {
+ if (endpointBuilder_ == null) {
+ endpoint_ = builderForValue.build();
+ onChanged();
+ } else {
+ endpointBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public Builder mergeEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+ if (endpointBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001) &&
+ endpoint_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
+ endpoint_ =
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(endpoint_).mergeFrom(value).buildPartial();
+ } else {
+ endpoint_ = value;
+ }
+ onChanged();
+ } else {
+ endpointBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public Builder clearEndpoint() {
+ if (endpointBuilder_ == null) {
+ endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+ onChanged();
+ } else {
+ endpointBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getEndpointBuilder() {
+ bitField0_ |= 0x00000001;
+ onChanged();
+ return getEndpointFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
+ if (endpointBuilder_ != null) {
+ return endpointBuilder_.getMessageOrBuilder();
+ } else {
+ return endpoint_;
+ }
+ }
+ /**
+ * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>
+ getEndpointFieldBuilder() {
+ if (endpointBuilder_ == null) {
+ endpointBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>(
+ endpoint_,
+ getParentForChildren(),
+ isClean());
+ endpoint_ = null;
+ }
+ return endpointBuilder_;
+ }
+
+ // optional int32 queue_length = 2;
+ private int queueLength_ ;
+ /**
+ * <code>optional int32 queue_length = 2;</code>
+ */
+ public boolean hasQueueLength() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional int32 queue_length = 2;</code>
+ */
+ public int getQueueLength() {
+ return queueLength_;
+ }
+ /**
+ * <code>optional int32 queue_length = 2;</code>
+ */
+ public Builder setQueueLength(int value) {
+ bitField0_ |= 0x00000002;
+ queueLength_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int32 queue_length = 2;</code>
+ */
+ public Builder clearQueueLength() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ queueLength_ = 0;
+ onChanged();
+ return this;
+ }
+
+ // optional int64 report_time = 3;
+ private long reportTime_ ;
+ /**
+ * <code>optional int64 report_time = 3;</code>
+ */
+ public boolean hasReportTime() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional int64 report_time = 3;</code>
+ */
+ public long getReportTime() {
+ return reportTime_;
+ }
+ /**
+ * <code>optional int64 report_time = 3;</code>
+ */
+ public Builder setReportTime(long value) {
+ bitField0_ |= 0x00000004;
+ reportTime_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int64 report_time = 3;</code>
+ */
+ public Builder clearReportTime() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ reportTime_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:exec.bit.control.WorkQueueStatus)
+ }
+
+ static {
+ defaultInstance = new WorkQueueStatus(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:exec.bit.control.WorkQueueStatus)
+ }
+
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_exec_bit_control_BitControlHandshake_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_exec_bit_control_BitControlHandshake_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_exec_bit_control_BitStatus_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_exec_bit_control_BitStatus_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_exec_bit_control_FragmentStatus_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_exec_bit_control_FragmentStatus_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_exec_bit_control_PlanFragment_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_exec_bit_control_PlanFragment_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_exec_bit_control_WorkQueueStatus_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_exec_bit_control_WorkQueueStatus_fieldAccessorTable;
+
+ public static com.google.protobuf.Descriptors.FileDescriptor
+ getDescriptor() {
+ return descriptor;
+ }
+ private static com.google.protobuf.Descriptors.FileDescriptor
+ descriptor;
+ static {
+ java.lang.String[] descriptorData = {
+ "\n\020BitControl.proto\022\020exec.bit.control\032\025Ex" +
+ "ecutionProtos.proto\032\022Coordination.proto\032" +
+ "\023UserBitShared.proto\"\213\001\n\023BitControlHands" +
+ "hake\022\023\n\013rpc_version\030\001 \001(\005\0225\n\007channel\030\002 \001" +
+ "(\0162\027.exec.shared.RpcChannel:\013BIT_CONTROL" +
+ "\022(\n\010endpoint\030\003 \001(\0132\026.exec.DrillbitEndpoi" +
+ "nt\"F\n\tBitStatus\0229\n\017fragment_status\030\001 \003(\013" +
+ "2 .exec.bit.control.FragmentStatus\"\261\003\n\016F" +
+ "ragmentStatus\022\022\n\nmemory_use\030\001 \001(\003\022\031\n\021bat" +
+ "ches_completed\030\002 \001(\003\022\031\n\021records_complete",
+ "d\030\003 \001(\003\022\'\n\037estimated_completion_percenta" +
+ "ge\030\004 \001(\005\022=\n\005state\030\005 \001(\0162..exec.bit.contr" +
+ "ol.FragmentStatus.FragmentState\022\026\n\016data_" +
+ "processed\030\006 \001(\003\022(\n\006handle\030\007 \001(\0132\030.exec.b" +
+ "it.FragmentHandle\022(\n\005error\030\010 \001(\0132\031.exec." +
+ "shared.DrillPBError\022\024\n\014running_time\030\t \001(" +
+ "\003\"k\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAIT" +
+ "ING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHE" +
+ "D\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\"\310\002\n\014PlanF" +
+ "ragment\022(\n\006handle\030\001 \001(\0132\030.exec.bit.Fragm",
+ "entHandle\022\024\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_c" +
+ "ost\030\005 \001(\002\022\021\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_c" +
+ "ost\030\007 \001(\002\022\025\n\rfragment_json\030\010 \001(\t\022*\n\nassi" +
+ "gnment\030\n \001(\0132\026.exec.DrillbitEndpoint\022\025\n\r" +
+ "leaf_fragment\030\t \001(\010\022\'\n\007foreman\030\013 \001(\0132\026.e" +
+ "xec.DrillbitEndpoint\022\035\n\013mem_initial\030\014 \001(" +
+ "\003:\01020000000\022\034\n\007mem_max\030\r \001(\003:\0132000000000" +
+ "0\"f\n\017WorkQueueStatus\022(\n\010endpoint\030\001 \001(\0132\026" +
+ ".exec.DrillbitEndpoint\022\024\n\014queue_length\030\002" +
+ " \001(\005\022\023\n\013report_time\030\003 \001(\003*\332\001\n\007RpcType\022\r\n",
+ "\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\033\n\027RE" +
+ "Q_INIATILIZE_FRAGMENT\020\003\022\027\n\023REQ_CANCEL_FR" +
+ "AGMENT\020\006\022\027\n\023REQ_FRAGMENT_STATUS\020\007\022\022\n\016REQ" +
+ "_BIT_STATUS\020\010\022\030\n\024RESP_FRAGMENT_HANDLE\020\t\022" +
+ "\030\n\024RESP_FRAGMENT_STATUS\020\n\022\023\n\017RESP_BIT_ST" +
+ "ATUS\020\013B+\n\033org.apache.drill.exec.protoB\nB" +
+ "itControlH\001"
+ };
+ com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+ new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+ public com.google.protobuf.ExtensionRegistry assignDescriptors(
+ com.google.protobuf.Descriptors.FileDescriptor root) {
+ descriptor = root;
+ internal_static_exec_bit_control_BitControlHandshake_descriptor =
+ getDescriptor().getMessageTypes().get(0);
+ internal_static_exec_bit_control_BitControlHandshake_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_exec_bit_control_BitControlHandshake_descriptor,
+ new java.lang.String[] { "RpcVersion", "Channel", "Endpoint", });
+ internal_static_exec_bit_control_BitStatus_descriptor =
+ getDescriptor().getMessageTypes().get(1);
+ internal_static_exec_bit_control_BitStatus_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_exec_bit_control_BitStatus_descriptor,
+ new java.lang.String[] { "FragmentStatus", });
+ internal_static_exec_bit_control_FragmentStatus_descriptor =
+ getDescriptor().getMessageTypes().get(2);
+ internal_static_exec_bit_control_FragmentStatus_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_exec_bit_control_FragmentStatus_descriptor,
+ new java.lang.String[] { "MemoryUse", "BatchesCompleted", "RecordsCompleted", "EstimatedCompletionPercentage", "State", "DataProcessed", "Handle", "Error", "RunningTime", });
+ internal_static_exec_bit_control_PlanFragment_descriptor =
+ getDescriptor().getMessageTypes().get(3);
+ internal_static_exec_bit_control_PlanFragment_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_exec_bit_control_PlanFragment_descriptor,
+ new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "Assignment", "LeafFragment", "Foreman", "MemInitial", "MemMax", });
+ internal_static_exec_bit_control_WorkQueueStatus_descriptor =
+ getDescriptor().getMessageTypes().get(4);
+ internal_static_exec_bit_control_WorkQueueStatus_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_exec_bit_control_WorkQueueStatus_descriptor,
+ new java.lang.String[] { "Endpoint", "QueueLength", "ReportTime", });
+ return null;
+ }
+ };
+ com.google.protobuf.Descriptors.FileDescriptor
+ .internalBuildGeneratedFileFrom(descriptorData,
+ new com.google.protobuf.Descriptors.FileDescriptor[] {
+ org.apache.drill.exec.proto.ExecProtos.getDescriptor(),
+ org.apache.drill.exec.proto.CoordinationProtos.getDescriptor(),
+ org.apache.drill.exec.proto.UserBitShared.getDescriptor(),
+ }, assigner);
+ }
+
+ // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitData.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitData.java
new file mode 100644
index 000000000..78343e6e6
--- /dev/null
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitData.java
@@ -0,0 +1,2228 @@
+/**
+ * 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.
+ */
+// Generated by the protocol buffer compiler. DO NOT EDIT!
+// source: BitData.proto
+
+package org.apache.drill.exec.proto;
+
+public final class BitData {
+ private BitData() {}
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistry registry) {
+ }
+ /**
+ * Protobuf enum {@code exec.bit.data.RpcType}
+ */
+ public enum RpcType
+ implements com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * <code>HANDSHAKE = 0;</code>
+ */
+ HANDSHAKE(0, 0),
+ /**
+ * <code>ACK = 1;</code>
+ */
+ ACK(1, 1),
+ /**
+ * <code>GOODBYE = 2;</code>
+ */
+ GOODBYE(2, 2),
+ /**
+ * <code>REQ_RECORD_BATCH = 3;</code>
+ *
+ * <pre>
+ * send record batch, returns Ack
+ * </pre>
+ */
+ REQ_RECORD_BATCH(3, 3),
+ ;
+
+ /**
+ * <code>HANDSHAKE = 0;</code>
+ */
+ public static final int HANDSHAKE_VALUE = 0;
+ /**
+ * <code>ACK = 1;</code>
+ */
+ public static final int ACK_VALUE = 1;
+ /**
+ * <code>GOODBYE = 2;</code>
+ */
+ public static final int GOODBYE_VALUE = 2;
+ /**
+ * <code>REQ_RECORD_BATCH = 3;</code>
+ *
+ * <pre>
+ * send record batch, returns Ack
+ * </pre>
+ */
+ public static final int REQ_RECORD_BATCH_VALUE = 3;
+
+
+ public final int getNumber() { return value; }
+
+ public static RpcType valueOf(int value) {
+ switch (value) {
+ case 0: return HANDSHAKE;
+ case 1: return ACK;
+ case 2: return GOODBYE;
+ case 3: return REQ_RECORD_BATCH;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap<RpcType>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap<RpcType>
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap<RpcType>() {
+ public RpcType findValueByNumber(int number) {
+ return RpcType.valueOf(number);
+ }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(index);
+ }
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitData.getDescriptor().getEnumTypes().get(0);
+ }
+
+ private static final RpcType[] VALUES = values();
+
+ public static RpcType valueOf(
+ com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+
+ private final int index;
+ private final int value;
+
+ private RpcType(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:exec.bit.data.RpcType)
+ }
+
+ public interface BitClientHandshakeOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // optional int32 rpc_version = 1;
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ boolean hasRpcVersion();
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ int getRpcVersion();
+
+ // optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];</code>
+ */
+ boolean hasChannel();
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];</code>
+ */
+ org.apache.drill.exec.proto.UserBitShared.RpcChannel getChannel();
+
+ // optional .exec.bit.FragmentHandle handle = 3;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ boolean hasHandle();
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle();
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder();
+ }
+ /**
+ * Protobuf type {@code exec.bit.data.BitClientHandshake}
+ */
+ public static final class BitClientHandshake extends
+ com.google.protobuf.GeneratedMessage
+ implements BitClientHandshakeOrBuilder {
+ // Use BitClientHandshake.newBuilder() to construct.
+ private BitClientHandshake(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private BitClientHandshake(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final BitClientHandshake defaultInstance;
+ public static BitClientHandshake getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public BitClientHandshake getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private BitClientHandshake(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 8: {
+ bitField0_ |= 0x00000001;
+ rpcVersion_ = input.readInt32();
+ break;
+ }
+ case 16: {
+ int rawValue = input.readEnum();
+ org.apache.drill.exec.proto.UserBitShared.RpcChannel value = org.apache.drill.exec.proto.UserBitShared.RpcChannel.valueOf(rawValue);
+ if (value == null) {
+ unknownFields.mergeVarintField(2, rawValue);
+ } else {
+ bitField0_ |= 0x00000002;
+ channel_ = value;
+ }
+ break;
+ }
+ case 26: {
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ subBuilder = handle_.toBuilder();
+ }
+ handle_ = input.readMessage(org.apache.drill.exec.proto.ExecProtos.FragmentHandle.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(handle_);
+ handle_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000004;
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitClientHandshake_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitClientHandshake_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitData.BitClientHandshake.class, org.apache.drill.exec.proto.BitData.BitClientHandshake.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<BitClientHandshake> PARSER =
+ new com.google.protobuf.AbstractParser<BitClientHandshake>() {
+ public BitClientHandshake parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new BitClientHandshake(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<BitClientHandshake> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // optional int32 rpc_version = 1;
+ public static final int RPC_VERSION_FIELD_NUMBER = 1;
+ private int rpcVersion_;
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public boolean hasRpcVersion() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public int getRpcVersion() {
+ return rpcVersion_;
+ }
+
+ // optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];
+ public static final int CHANNEL_FIELD_NUMBER = 2;
+ private org.apache.drill.exec.proto.UserBitShared.RpcChannel channel_;
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];</code>
+ */
+ public boolean hasChannel() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RpcChannel getChannel() {
+ return channel_;
+ }
+
+ // optional .exec.bit.FragmentHandle handle = 3;
+ public static final int HANDLE_FIELD_NUMBER = 3;
+ private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public boolean hasHandle() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
+ return handle_;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
+ return handle_;
+ }
+
+ private void initFields() {
+ rpcVersion_ = 0;
+ channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.BIT_DATA;
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeInt32(1, rpcVersion_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeEnum(2, channel_.getNumber());
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeMessage(3, handle_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt32Size(1, rpcVersion_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(2, channel_.getNumber());
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(3, handle_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitClientHandshake parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.drill.exec.proto.BitData.BitClientHandshake prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code exec.bit.data.BitClientHandshake}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.drill.exec.proto.BitData.BitClientHandshakeOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitClientHandshake_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitClientHandshake_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitData.BitClientHandshake.class, org.apache.drill.exec.proto.BitData.BitClientHandshake.Builder.class);
+ }
+
+ // Construct using org.apache.drill.exec.proto.BitData.BitClientHandshake.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getHandleFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ rpcVersion_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.BIT_DATA;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ if (handleBuilder_ == null) {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ } else {
+ handleBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000004);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitClientHandshake_descriptor;
+ }
+
+ public org.apache.drill.exec.proto.BitData.BitClientHandshake getDefaultInstanceForType() {
+ return org.apache.drill.exec.proto.BitData.BitClientHandshake.getDefaultInstance();
+ }
+
+ public org.apache.drill.exec.proto.BitData.BitClientHandshake build() {
+ org.apache.drill.exec.proto.BitData.BitClientHandshake result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.drill.exec.proto.BitData.BitClientHandshake buildPartial() {
+ org.apache.drill.exec.proto.BitData.BitClientHandshake result = new org.apache.drill.exec.proto.BitData.BitClientHandshake(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.rpcVersion_ = rpcVersion_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.channel_ = channel_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ if (handleBuilder_ == null) {
+ result.handle_ = handle_;
+ } else {
+ result.handle_ = handleBuilder_.build();
+ }
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.drill.exec.proto.BitData.BitClientHandshake) {
+ return mergeFrom((org.apache.drill.exec.proto.BitData.BitClientHandshake)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.drill.exec.proto.BitData.BitClientHandshake other) {
+ if (other == org.apache.drill.exec.proto.BitData.BitClientHandshake.getDefaultInstance()) return this;
+ if (other.hasRpcVersion()) {
+ setRpcVersion(other.getRpcVersion());
+ }
+ if (other.hasChannel()) {
+ setChannel(other.getChannel());
+ }
+ if (other.hasHandle()) {
+ mergeHandle(other.getHandle());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.drill.exec.proto.BitData.BitClientHandshake parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.drill.exec.proto.BitData.BitClientHandshake) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // optional int32 rpc_version = 1;
+ private int rpcVersion_ ;
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public boolean hasRpcVersion() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public int getRpcVersion() {
+ return rpcVersion_;
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public Builder setRpcVersion(int value) {
+ bitField0_ |= 0x00000001;
+ rpcVersion_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public Builder clearRpcVersion() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ rpcVersion_ = 0;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];
+ private org.apache.drill.exec.proto.UserBitShared.RpcChannel channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.BIT_DATA;
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];</code>
+ */
+ public boolean hasChannel() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RpcChannel getChannel() {
+ return channel_;
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];</code>
+ */
+ public Builder setChannel(org.apache.drill.exec.proto.UserBitShared.RpcChannel value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000002;
+ channel_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 2 [default = BIT_DATA];</code>
+ */
+ public Builder clearChannel() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.BIT_DATA;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.bit.FragmentHandle handle = 3;
+ private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder> handleBuilder_;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public boolean hasHandle() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
+ if (handleBuilder_ == null) {
+ return handle_;
+ } else {
+ return handleBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public Builder setHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
+ if (handleBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ handle_ = value;
+ onChanged();
+ } else {
+ handleBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000004;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public Builder setHandle(
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder builderForValue) {
+ if (handleBuilder_ == null) {
+ handle_ = builderForValue.build();
+ onChanged();
+ } else {
+ handleBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000004;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public Builder mergeHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
+ if (handleBuilder_ == null) {
+ if (((bitField0_ & 0x00000004) == 0x00000004) &&
+ handle_ != org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance()) {
+ handle_ =
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.newBuilder(handle_).mergeFrom(value).buildPartial();
+ } else {
+ handle_ = value;
+ }
+ onChanged();
+ } else {
+ handleBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000004;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public Builder clearHandle() {
+ if (handleBuilder_ == null) {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ onChanged();
+ } else {
+ handleBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000004);
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder getHandleBuilder() {
+ bitField0_ |= 0x00000004;
+ onChanged();
+ return getHandleFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
+ if (handleBuilder_ != null) {
+ return handleBuilder_.getMessageOrBuilder();
+ } else {
+ return handle_;
+ }
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 3;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>
+ getHandleFieldBuilder() {
+ if (handleBuilder_ == null) {
+ handleBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>(
+ handle_,
+ getParentForChildren(),
+ isClean());
+ handle_ = null;
+ }
+ return handleBuilder_;
+ }
+
+ // @@protoc_insertion_point(builder_scope:exec.bit.data.BitClientHandshake)
+ }
+
+ static {
+ defaultInstance = new BitClientHandshake(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:exec.bit.data.BitClientHandshake)
+ }
+
+ public interface BitServerHandshakeOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // optional int32 rpc_version = 1;
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ boolean hasRpcVersion();
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ int getRpcVersion();
+ }
+ /**
+ * Protobuf type {@code exec.bit.data.BitServerHandshake}
+ */
+ public static final class BitServerHandshake extends
+ com.google.protobuf.GeneratedMessage
+ implements BitServerHandshakeOrBuilder {
+ // Use BitServerHandshake.newBuilder() to construct.
+ private BitServerHandshake(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private BitServerHandshake(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final BitServerHandshake defaultInstance;
+ public static BitServerHandshake getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public BitServerHandshake getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private BitServerHandshake(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 8: {
+ bitField0_ |= 0x00000001;
+ rpcVersion_ = input.readInt32();
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitServerHandshake_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitServerHandshake_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitData.BitServerHandshake.class, org.apache.drill.exec.proto.BitData.BitServerHandshake.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<BitServerHandshake> PARSER =
+ new com.google.protobuf.AbstractParser<BitServerHandshake>() {
+ public BitServerHandshake parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new BitServerHandshake(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<BitServerHandshake> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // optional int32 rpc_version = 1;
+ public static final int RPC_VERSION_FIELD_NUMBER = 1;
+ private int rpcVersion_;
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public boolean hasRpcVersion() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public int getRpcVersion() {
+ return rpcVersion_;
+ }
+
+ private void initFields() {
+ rpcVersion_ = 0;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeInt32(1, rpcVersion_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt32Size(1, rpcVersion_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitData.BitServerHandshake parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.drill.exec.proto.BitData.BitServerHandshake prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code exec.bit.data.BitServerHandshake}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.drill.exec.proto.BitData.BitServerHandshakeOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitServerHandshake_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitServerHandshake_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitData.BitServerHandshake.class, org.apache.drill.exec.proto.BitData.BitServerHandshake.Builder.class);
+ }
+
+ // Construct using org.apache.drill.exec.proto.BitData.BitServerHandshake.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ rpcVersion_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_BitServerHandshake_descriptor;
+ }
+
+ public org.apache.drill.exec.proto.BitData.BitServerHandshake getDefaultInstanceForType() {
+ return org.apache.drill.exec.proto.BitData.BitServerHandshake.getDefaultInstance();
+ }
+
+ public org.apache.drill.exec.proto.BitData.BitServerHandshake build() {
+ org.apache.drill.exec.proto.BitData.BitServerHandshake result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.drill.exec.proto.BitData.BitServerHandshake buildPartial() {
+ org.apache.drill.exec.proto.BitData.BitServerHandshake result = new org.apache.drill.exec.proto.BitData.BitServerHandshake(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.rpcVersion_ = rpcVersion_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.drill.exec.proto.BitData.BitServerHandshake) {
+ return mergeFrom((org.apache.drill.exec.proto.BitData.BitServerHandshake)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.drill.exec.proto.BitData.BitServerHandshake other) {
+ if (other == org.apache.drill.exec.proto.BitData.BitServerHandshake.getDefaultInstance()) return this;
+ if (other.hasRpcVersion()) {
+ setRpcVersion(other.getRpcVersion());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.drill.exec.proto.BitData.BitServerHandshake parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.drill.exec.proto.BitData.BitServerHandshake) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // optional int32 rpc_version = 1;
+ private int rpcVersion_ ;
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public boolean hasRpcVersion() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public int getRpcVersion() {
+ return rpcVersion_;
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public Builder setRpcVersion(int value) {
+ bitField0_ |= 0x00000001;
+ rpcVersion_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int32 rpc_version = 1;</code>
+ */
+ public Builder clearRpcVersion() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ rpcVersion_ = 0;
+ onChanged();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:exec.bit.data.BitServerHandshake)
+ }
+
+ static {
+ defaultInstance = new BitServerHandshake(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:exec.bit.data.BitServerHandshake)
+ }
+
+ public interface FragmentRecordBatchOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // optional .exec.bit.FragmentHandle handle = 1;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ boolean hasHandle();
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle();
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder();
+
+ // optional int32 sending_major_fragment_id = 2;
+ /**
+ * <code>optional int32 sending_major_fragment_id = 2;</code>
+ */
+ boolean hasSendingMajorFragmentId();
+ /**
+ * <code>optional int32 sending_major_fragment_id = 2;</code>
+ */
+ int getSendingMajorFragmentId();
+
+ // optional int32 sending_minor_fragment_id = 3;
+ /**
+ * <code>optional int32 sending_minor_fragment_id = 3;</code>
+ */
+ boolean hasSendingMinorFragmentId();
+ /**
+ * <code>optional int32 sending_minor_fragment_id = 3;</code>
+ */
+ int getSendingMinorFragmentId();
+
+ // optional .exec.shared.RecordBatchDef def = 4;
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ boolean hasDef();
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef();
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder();
+
+ // optional bool isLastBatch = 5;
+ /**
+ * <code>optional bool isLastBatch = 5;</code>
+ */
+ boolean hasIsLastBatch();
+ /**
+ * <code>optional bool isLastBatch = 5;</code>
+ */
+ boolean getIsLastBatch();
+ }
+ /**
+ * Protobuf type {@code exec.bit.data.FragmentRecordBatch}
+ */
+ public static final class FragmentRecordBatch extends
+ com.google.protobuf.GeneratedMessage
+ implements FragmentRecordBatchOrBuilder {
+ // Use FragmentRecordBatch.newBuilder() to construct.
+ private FragmentRecordBatch(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private FragmentRecordBatch(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final FragmentRecordBatch defaultInstance;
+ public static FragmentRecordBatch getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public FragmentRecordBatch getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private FragmentRecordBatch(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 10: {
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ subBuilder = handle_.toBuilder();
+ }
+ handle_ = input.readMessage(org.apache.drill.exec.proto.ExecProtos.FragmentHandle.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(handle_);
+ handle_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000001;
+ break;
+ }
+ case 16: {
+ bitField0_ |= 0x00000002;
+ sendingMajorFragmentId_ = input.readInt32();
+ break;
+ }
+ case 24: {
+ bitField0_ |= 0x00000004;
+ sendingMinorFragmentId_ = input.readInt32();
+ break;
+ }
+ case 34: {
+ org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ subBuilder = def_.toBuilder();
+ }
+ def_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(def_);
+ def_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000008;
+ break;
+ }
+ case 40: {
+ bitField0_ |= 0x00000010;
+ isLastBatch_ = input.readBool();
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_FragmentRecordBatch_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_FragmentRecordBatch_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitData.FragmentRecordBatch.class, org.apache.drill.exec.proto.BitData.FragmentRecordBatch.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<FragmentRecordBatch> PARSER =
+ new com.google.protobuf.AbstractParser<FragmentRecordBatch>() {
+ public FragmentRecordBatch parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new FragmentRecordBatch(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<FragmentRecordBatch> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // optional .exec.bit.FragmentHandle handle = 1;
+ public static final int HANDLE_FIELD_NUMBER = 1;
+ private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public boolean hasHandle() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
+ return handle_;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
+ return handle_;
+ }
+
+ // optional int32 sending_major_fragment_id = 2;
+ public static final int SENDING_MAJOR_FRAGMENT_ID_FIELD_NUMBER = 2;
+ private int sendingMajorFragmentId_;
+ /**
+ * <code>optional int32 sending_major_fragment_id = 2;</code>
+ */
+ public boolean hasSendingMajorFragmentId() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional int32 sending_major_fragment_id = 2;</code>
+ */
+ public int getSendingMajorFragmentId() {
+ return sendingMajorFragmentId_;
+ }
+
+ // optional int32 sending_minor_fragment_id = 3;
+ public static final int SENDING_MINOR_FRAGMENT_ID_FIELD_NUMBER = 3;
+ private int sendingMinorFragmentId_;
+ /**
+ * <code>optional int32 sending_minor_fragment_id = 3;</code>
+ */
+ public boolean hasSendingMinorFragmentId() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional int32 sending_minor_fragment_id = 3;</code>
+ */
+ public int getSendingMinorFragmentId() {
+ return sendingMinorFragmentId_;
+ }
+
+ // optional .exec.shared.RecordBatchDef def = 4;
+ public static final int DEF_FIELD_NUMBER = 4;
+ private org.apache.drill.exec.proto.UserBitShared.RecordBatchDef def_;
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public boolean hasDef() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef() {
+ return def_;
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder() {
+ return def_;
+ }
+
+ // optional bool isLastBatch = 5;
+ public static final int ISLASTBATCH_FIELD_NUMBER = 5;
+ private boolean isLastBatch_;
+ /**
+ * <code>optional bool isLastBatch = 5;</code>
+ */
+ public boolean hasIsLastBatch() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * <code>optional bool isLastBatch = 5;</code>
+ */
+ public boolean getIsLastBatch() {
+ return isLastBatch_;
+ }
+
+ private void initFields() {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ sendingMajorFragmentId_ = 0;
+ sendingMinorFragmentId_ = 0;
+ def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
+ isLastBatch_ = false;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeMessage(1, handle_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeInt32(2, sendingMajorFragmentId_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeInt32(3, sendingMinorFragmentId_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeMessage(4, def_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ output.writeBool(5, isLastBatch_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, handle_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt32Size(2, sendingMajorFragmentId_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt32Size(3, sendingMinorFragmentId_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(4, def_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(5, isLastBatch_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.drill.exec.proto.BitData.FragmentRecordBatch parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.drill.exec.proto.BitData.FragmentRecordBatch prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code exec.bit.data.FragmentRecordBatch}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.drill.exec.proto.BitData.FragmentRecordBatchOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_FragmentRecordBatch_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_FragmentRecordBatch_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.drill.exec.proto.BitData.FragmentRecordBatch.class, org.apache.drill.exec.proto.BitData.FragmentRecordBatch.Builder.class);
+ }
+
+ // Construct using org.apache.drill.exec.proto.BitData.FragmentRecordBatch.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getHandleFieldBuilder();
+ getDefFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ if (handleBuilder_ == null) {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ } else {
+ handleBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ sendingMajorFragmentId_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ sendingMinorFragmentId_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ if (defBuilder_ == null) {
+ def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
+ } else {
+ defBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000008);
+ isLastBatch_ = false;
+ bitField0_ = (bitField0_ & ~0x00000010);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.drill.exec.proto.BitData.internal_static_exec_bit_data_FragmentRecordBatch_descriptor;
+ }
+
+ public org.apache.drill.exec.proto.BitData.FragmentRecordBatch getDefaultInstanceForType() {
+ return org.apache.drill.exec.proto.BitData.FragmentRecordBatch.getDefaultInstance();
+ }
+
+ public org.apache.drill.exec.proto.BitData.FragmentRecordBatch build() {
+ org.apache.drill.exec.proto.BitData.FragmentRecordBatch result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.drill.exec.proto.BitData.FragmentRecordBatch buildPartial() {
+ org.apache.drill.exec.proto.BitData.FragmentRecordBatch result = new org.apache.drill.exec.proto.BitData.FragmentRecordBatch(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ if (handleBuilder_ == null) {
+ result.handle_ = handle_;
+ } else {
+ result.handle_ = handleBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.sendingMajorFragmentId_ = sendingMajorFragmentId_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.sendingMinorFragmentId_ = sendingMinorFragmentId_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ if (defBuilder_ == null) {
+ result.def_ = def_;
+ } else {
+ result.def_ = defBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+ to_bitField0_ |= 0x00000010;
+ }
+ result.isLastBatch_ = isLastBatch_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.drill.exec.proto.BitData.FragmentRecordBatch) {
+ return mergeFrom((org.apache.drill.exec.proto.BitData.FragmentRecordBatch)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.drill.exec.proto.BitData.FragmentRecordBatch other) {
+ if (other == org.apache.drill.exec.proto.BitData.FragmentRecordBatch.getDefaultInstance()) return this;
+ if (other.hasHandle()) {
+ mergeHandle(other.getHandle());
+ }
+ if (other.hasSendingMajorFragmentId()) {
+ setSendingMajorFragmentId(other.getSendingMajorFragmentId());
+ }
+ if (other.hasSendingMinorFragmentId()) {
+ setSendingMinorFragmentId(other.getSendingMinorFragmentId());
+ }
+ if (other.hasDef()) {
+ mergeDef(other.getDef());
+ }
+ if (other.hasIsLastBatch()) {
+ setIsLastBatch(other.getIsLastBatch());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.drill.exec.proto.BitData.FragmentRecordBatch parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.drill.exec.proto.BitData.FragmentRecordBatch) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // optional .exec.bit.FragmentHandle handle = 1;
+ private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder> handleBuilder_;
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public boolean hasHandle() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
+ if (handleBuilder_ == null) {
+ return handle_;
+ } else {
+ return handleBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public Builder setHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
+ if (handleBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ handle_ = value;
+ onChanged();
+ } else {
+ handleBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public Builder setHandle(
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder builderForValue) {
+ if (handleBuilder_ == null) {
+ handle_ = builderForValue.build();
+ onChanged();
+ } else {
+ handleBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public Builder mergeHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
+ if (handleBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001) &&
+ handle_ != org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance()) {
+ handle_ =
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle.newBuilder(handle_).mergeFrom(value).buildPartial();
+ } else {
+ handle_ = value;
+ }
+ onChanged();
+ } else {
+ handleBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public Builder clearHandle() {
+ if (handleBuilder_ == null) {
+ handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
+ onChanged();
+ } else {
+ handleBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder getHandleBuilder() {
+ bitField0_ |= 0x00000001;
+ onChanged();
+ return getHandleFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
+ if (handleBuilder_ != null) {
+ return handleBuilder_.getMessageOrBuilder();
+ } else {
+ return handle_;
+ }
+ }
+ /**
+ * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>
+ getHandleFieldBuilder() {
+ if (handleBuilder_ == null) {
+ handleBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>(
+ handle_,
+ getParentForChildren(),
+ isClean());
+ handle_ = null;
+ }
+ return handleBuilder_;
+ }
+
+ // optional int32 sending_major_fragment_id = 2;
+ private int sendingMajorFragmentId_ ;
+ /**
+ * <code>optional int32 sending_major_fragment_id = 2;</code>
+ */
+ public boolean hasSendingMajorFragmentId() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional int32 sending_major_fragment_id = 2;</code>
+ */
+ public int getSendingMajorFragmentId() {
+ return sendingMajorFragmentId_;
+ }
+ /**
+ * <code>optional int32 sending_major_fragment_id = 2;</code>
+ */
+ public Builder setSendingMajorFragmentId(int value) {
+ bitField0_ |= 0x00000002;
+ sendingMajorFragmentId_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int32 sending_major_fragment_id = 2;</code>
+ */
+ public Builder clearSendingMajorFragmentId() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ sendingMajorFragmentId_ = 0;
+ onChanged();
+ return this;
+ }
+
+ // optional int32 sending_minor_fragment_id = 3;
+ private int sendingMinorFragmentId_ ;
+ /**
+ * <code>optional int32 sending_minor_fragment_id = 3;</code>
+ */
+ public boolean hasSendingMinorFragmentId() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional int32 sending_minor_fragment_id = 3;</code>
+ */
+ public int getSendingMinorFragmentId() {
+ return sendingMinorFragmentId_;
+ }
+ /**
+ * <code>optional int32 sending_minor_fragment_id = 3;</code>
+ */
+ public Builder setSendingMinorFragmentId(int value) {
+ bitField0_ |= 0x00000004;
+ sendingMinorFragmentId_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int32 sending_minor_fragment_id = 3;</code>
+ */
+ public Builder clearSendingMinorFragmentId() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ sendingMinorFragmentId_ = 0;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.shared.RecordBatchDef def = 4;
+ private org.apache.drill.exec.proto.UserBitShared.RecordBatchDef def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.UserBitShared.RecordBatchDef, org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder, org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder> defBuilder_;
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public boolean hasDef() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef() {
+ if (defBuilder_ == null) {
+ return def_;
+ } else {
+ return defBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public Builder setDef(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef value) {
+ if (defBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ def_ = value;
+ onChanged();
+ } else {
+ defBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000008;
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public Builder setDef(
+ org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder builderForValue) {
+ if (defBuilder_ == null) {
+ def_ = builderForValue.build();
+ onChanged();
+ } else {
+ defBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000008;
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public Builder mergeDef(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef value) {
+ if (defBuilder_ == null) {
+ if (((bitField0_ & 0x00000008) == 0x00000008) &&
+ def_ != org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance()) {
+ def_ =
+ org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.newBuilder(def_).mergeFrom(value).buildPartial();
+ } else {
+ def_ = value;
+ }
+ onChanged();
+ } else {
+ defBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000008;
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public Builder clearDef() {
+ if (defBuilder_ == null) {
+ def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
+ onChanged();
+ } else {
+ defBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000008);
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder getDefBuilder() {
+ bitField0_ |= 0x00000008;
+ onChanged();
+ return getDefFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder() {
+ if (defBuilder_ != null) {
+ return defBuilder_.getMessageOrBuilder();
+ } else {
+ return def_;
+ }
+ }
+ /**
+ * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.UserBitShared.RecordBatchDef, org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder, org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder>
+ getDefFieldBuilder() {
+ if (defBuilder_ == null) {
+ defBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.drill.exec.proto.UserBitShared.RecordBatchDef, org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder, org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder>(
+ def_,
+ getParentForChildren(),
+ isClean());
+ def_ = null;
+ }
+ return defBuilder_;
+ }
+
+ // optional bool isLastBatch = 5;
+ private boolean isLastBatch_ ;
+ /**
+ * <code>optional bool isLastBatch = 5;</code>
+ */
+ public boolean hasIsLastBatch() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * <code>optional bool isLastBatch = 5;</code>
+ */
+ public boolean getIsLastBatch() {
+ return isLastBatch_;
+ }
+ /**
+ * <code>optional bool isLastBatch = 5;</code>
+ */
+ public Builder setIsLastBatch(boolean value) {
+ bitField0_ |= 0x00000010;
+ isLastBatch_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bool isLastBatch = 5;</code>
+ */
+ public Builder clearIsLastBatch() {
+ bitField0_ = (bitField0_ & ~0x00000010);
+ isLastBatch_ = false;
+ onChanged();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:exec.bit.data.FragmentRecordBatch)
+ }
+
+ static {
+ defaultInstance = new FragmentRecordBatch(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:exec.bit.data.FragmentRecordBatch)
+ }
+
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_exec_bit_data_BitClientHandshake_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_exec_bit_data_BitClientHandshake_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_exec_bit_data_BitServerHandshake_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_exec_bit_data_BitServerHandshake_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_exec_bit_data_FragmentRecordBatch_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_exec_bit_data_FragmentRecordBatch_fieldAccessorTable;
+
+ public static com.google.protobuf.Descriptors.FileDescriptor
+ getDescriptor() {
+ return descriptor;
+ }
+ private static com.google.protobuf.Descriptors.FileDescriptor
+ descriptor;
+ static {
+ java.lang.String[] descriptorData = {
+ "\n\rBitData.proto\022\rexec.bit.data\032\025Executio" +
+ "nProtos.proto\032\022Coordination.proto\032\023UserB" +
+ "itShared.proto\"\207\001\n\022BitClientHandshake\022\023\n" +
+ "\013rpc_version\030\001 \001(\005\0222\n\007channel\030\002 \001(\0162\027.ex" +
+ "ec.shared.RpcChannel:\010BIT_DATA\022(\n\006handle" +
+ "\030\003 \001(\0132\030.exec.bit.FragmentHandle\")\n\022BitS" +
+ "erverHandshake\022\023\n\013rpc_version\030\001 \001(\005\"\304\001\n\023" +
+ "FragmentRecordBatch\022(\n\006handle\030\001 \001(\0132\030.ex" +
+ "ec.bit.FragmentHandle\022!\n\031sending_major_f" +
+ "ragment_id\030\002 \001(\005\022!\n\031sending_minor_fragme",
+ "nt_id\030\003 \001(\005\022(\n\003def\030\004 \001(\0132\033.exec.shared.R" +
+ "ecordBatchDef\022\023\n\013isLastBatch\030\005 \001(\010*D\n\007Rp" +
+ "cType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE" +
+ "\020\002\022\024\n\020REQ_RECORD_BATCH\020\003B(\n\033org.apache.d" +
+ "rill.exec.protoB\007BitDataH\001"
+ };
+ com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+ new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+ public com.google.protobuf.ExtensionRegistry assignDescriptors(
+ com.google.protobuf.Descriptors.FileDescriptor root) {
+ descriptor = root;
+ internal_static_exec_bit_data_BitClientHandshake_descriptor =
+ getDescriptor().getMessageTypes().get(0);
+ internal_static_exec_bit_data_BitClientHandshake_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_exec_bit_data_BitClientHandshake_descriptor,
+ new java.lang.String[] { "RpcVersion", "Channel", "Handle", });
+ internal_static_exec_bit_data_BitServerHandshake_descriptor =
+ getDescriptor().getMessageTypes().get(1);
+ internal_static_exec_bit_data_BitServerHandshake_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_exec_bit_data_BitServerHandshake_descriptor,
+ new java.lang.String[] { "RpcVersion", });
+ internal_static_exec_bit_data_FragmentRecordBatch_descriptor =
+ getDescriptor().getMessageTypes().get(2);
+ internal_static_exec_bit_data_FragmentRecordBatch_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_exec_bit_data_FragmentRecordBatch_descriptor,
+ new java.lang.String[] { "Handle", "SendingMajorFragmentId", "SendingMinorFragmentId", "Def", "IsLastBatch", });
+ return null;
+ }
+ };
+ com.google.protobuf.Descriptors.FileDescriptor
+ .internalBuildGeneratedFileFrom(descriptorData,
+ new com.google.protobuf.Descriptors.FileDescriptor[] {
+ org.apache.drill.exec.proto.ExecProtos.getDescriptor(),
+ org.apache.drill.exec.proto.CoordinationProtos.getDescriptor(),
+ org.apache.drill.exec.proto.UserBitShared.getDescriptor(),
+ }, assigner);
+ }
+
+ // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/CoordinationProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/CoordinationProtos.java
index 01b8889b4..177e56018 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/CoordinationProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/CoordinationProtos.java
@@ -53,27 +53,37 @@ public final class CoordinationProtos {
*/
int getUserPort();
- // optional int32 bit_port = 3;
+ // optional int32 control_port = 3;
/**
- * <code>optional int32 bit_port = 3;</code>
+ * <code>optional int32 control_port = 3;</code>
*/
- boolean hasBitPort();
+ boolean hasControlPort();
/**
- * <code>optional int32 bit_port = 3;</code>
+ * <code>optional int32 control_port = 3;</code>
*/
- int getBitPort();
+ int getControlPort();
- // optional .exec.Roles roles = 4;
+ // optional int32 data_port = 4;
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional int32 data_port = 4;</code>
+ */
+ boolean hasDataPort();
+ /**
+ * <code>optional int32 data_port = 4;</code>
+ */
+ int getDataPort();
+
+ // optional .exec.Roles roles = 5;
+ /**
+ * <code>optional .exec.Roles roles = 5;</code>
*/
boolean hasRoles();
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
org.apache.drill.exec.proto.CoordinationProtos.Roles getRoles();
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
org.apache.drill.exec.proto.CoordinationProtos.RolesOrBuilder getRolesOrBuilder();
}
@@ -140,12 +150,17 @@ public final class CoordinationProtos {
}
case 24: {
bitField0_ |= 0x00000004;
- bitPort_ = input.readInt32();
+ controlPort_ = input.readInt32();
+ break;
+ }
+ case 32: {
+ bitField0_ |= 0x00000008;
+ dataPort_ = input.readInt32();
break;
}
- case 34: {
+ case 42: {
org.apache.drill.exec.proto.CoordinationProtos.Roles.Builder subBuilder = null;
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
subBuilder = roles_.toBuilder();
}
roles_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.Roles.PARSER, extensionRegistry);
@@ -153,7 +168,7 @@ public final class CoordinationProtos {
subBuilder.mergeFrom(roles_);
roles_ = subBuilder.buildPartial();
}
- bitField0_ |= 0x00000008;
+ bitField0_ |= 0x00000010;
break;
}
}
@@ -255,39 +270,55 @@ public final class CoordinationProtos {
return userPort_;
}
- // optional int32 bit_port = 3;
- public static final int BIT_PORT_FIELD_NUMBER = 3;
- private int bitPort_;
+ // optional int32 control_port = 3;
+ public static final int CONTROL_PORT_FIELD_NUMBER = 3;
+ private int controlPort_;
/**
- * <code>optional int32 bit_port = 3;</code>
+ * <code>optional int32 control_port = 3;</code>
*/
- public boolean hasBitPort() {
+ public boolean hasControlPort() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
/**
- * <code>optional int32 bit_port = 3;</code>
+ * <code>optional int32 control_port = 3;</code>
*/
- public int getBitPort() {
- return bitPort_;
+ public int getControlPort() {
+ return controlPort_;
}
- // optional .exec.Roles roles = 4;
- public static final int ROLES_FIELD_NUMBER = 4;
+ // optional int32 data_port = 4;
+ public static final int DATA_PORT_FIELD_NUMBER = 4;
+ private int dataPort_;
+ /**
+ * <code>optional int32 data_port = 4;</code>
+ */
+ public boolean hasDataPort() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional int32 data_port = 4;</code>
+ */
+ public int getDataPort() {
+ return dataPort_;
+ }
+
+ // optional .exec.Roles roles = 5;
+ public static final int ROLES_FIELD_NUMBER = 5;
private org.apache.drill.exec.proto.CoordinationProtos.Roles roles_;
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public boolean hasRoles() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
+ return ((bitField0_ & 0x00000010) == 0x00000010);
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public org.apache.drill.exec.proto.CoordinationProtos.Roles getRoles() {
return roles_;
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public org.apache.drill.exec.proto.CoordinationProtos.RolesOrBuilder getRolesOrBuilder() {
return roles_;
@@ -296,7 +327,8 @@ public final class CoordinationProtos {
private void initFields() {
address_ = "";
userPort_ = 0;
- bitPort_ = 0;
+ controlPort_ = 0;
+ dataPort_ = 0;
roles_ = org.apache.drill.exec.proto.CoordinationProtos.Roles.getDefaultInstance();
}
private byte memoizedIsInitialized = -1;
@@ -318,10 +350,13 @@ public final class CoordinationProtos {
output.writeInt32(2, userPort_);
}
if (((bitField0_ & 0x00000004) == 0x00000004)) {
- output.writeInt32(3, bitPort_);
+ output.writeInt32(3, controlPort_);
}
if (((bitField0_ & 0x00000008) == 0x00000008)) {
- output.writeMessage(4, roles_);
+ output.writeInt32(4, dataPort_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ output.writeMessage(5, roles_);
}
getUnknownFields().writeTo(output);
}
@@ -342,11 +377,15 @@ public final class CoordinationProtos {
}
if (((bitField0_ & 0x00000004) == 0x00000004)) {
size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(3, bitPort_);
+ .computeInt32Size(3, controlPort_);
}
if (((bitField0_ & 0x00000008) == 0x00000008)) {
size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(4, roles_);
+ .computeInt32Size(4, dataPort_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(5, roles_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
@@ -469,14 +508,16 @@ public final class CoordinationProtos {
bitField0_ = (bitField0_ & ~0x00000001);
userPort_ = 0;
bitField0_ = (bitField0_ & ~0x00000002);
- bitPort_ = 0;
+ controlPort_ = 0;
bitField0_ = (bitField0_ & ~0x00000004);
+ dataPort_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000008);
if (rolesBuilder_ == null) {
roles_ = org.apache.drill.exec.proto.CoordinationProtos.Roles.getDefaultInstance();
} else {
rolesBuilder_.clear();
}
- bitField0_ = (bitField0_ & ~0x00000008);
+ bitField0_ = (bitField0_ & ~0x00000010);
return this;
}
@@ -516,10 +557,14 @@ public final class CoordinationProtos {
if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
to_bitField0_ |= 0x00000004;
}
- result.bitPort_ = bitPort_;
+ result.controlPort_ = controlPort_;
if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
to_bitField0_ |= 0x00000008;
}
+ result.dataPort_ = dataPort_;
+ if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+ to_bitField0_ |= 0x00000010;
+ }
if (rolesBuilder_ == null) {
result.roles_ = roles_;
} else {
@@ -549,8 +594,11 @@ public final class CoordinationProtos {
if (other.hasUserPort()) {
setUserPort(other.getUserPort());
}
- if (other.hasBitPort()) {
- setBitPort(other.getBitPort());
+ if (other.hasControlPort()) {
+ setControlPort(other.getControlPort());
+ }
+ if (other.hasDataPort()) {
+ setDataPort(other.getDataPort());
}
if (other.hasRoles()) {
mergeRoles(other.getRoles());
@@ -689,51 +737,84 @@ public final class CoordinationProtos {
return this;
}
- // optional int32 bit_port = 3;
- private int bitPort_ ;
+ // optional int32 control_port = 3;
+ private int controlPort_ ;
/**
- * <code>optional int32 bit_port = 3;</code>
+ * <code>optional int32 control_port = 3;</code>
*/
- public boolean hasBitPort() {
+ public boolean hasControlPort() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
/**
- * <code>optional int32 bit_port = 3;</code>
+ * <code>optional int32 control_port = 3;</code>
*/
- public int getBitPort() {
- return bitPort_;
+ public int getControlPort() {
+ return controlPort_;
}
/**
- * <code>optional int32 bit_port = 3;</code>
+ * <code>optional int32 control_port = 3;</code>
*/
- public Builder setBitPort(int value) {
+ public Builder setControlPort(int value) {
bitField0_ |= 0x00000004;
- bitPort_ = value;
+ controlPort_ = value;
onChanged();
return this;
}
/**
- * <code>optional int32 bit_port = 3;</code>
+ * <code>optional int32 control_port = 3;</code>
*/
- public Builder clearBitPort() {
+ public Builder clearControlPort() {
bitField0_ = (bitField0_ & ~0x00000004);
- bitPort_ = 0;
+ controlPort_ = 0;
onChanged();
return this;
}
- // optional .exec.Roles roles = 4;
+ // optional int32 data_port = 4;
+ private int dataPort_ ;
+ /**
+ * <code>optional int32 data_port = 4;</code>
+ */
+ public boolean hasDataPort() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional int32 data_port = 4;</code>
+ */
+ public int getDataPort() {
+ return dataPort_;
+ }
+ /**
+ * <code>optional int32 data_port = 4;</code>
+ */
+ public Builder setDataPort(int value) {
+ bitField0_ |= 0x00000008;
+ dataPort_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional int32 data_port = 4;</code>
+ */
+ public Builder clearDataPort() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ dataPort_ = 0;
+ onChanged();
+ return this;
+ }
+
+ // optional .exec.Roles roles = 5;
private org.apache.drill.exec.proto.CoordinationProtos.Roles roles_ = org.apache.drill.exec.proto.CoordinationProtos.Roles.getDefaultInstance();
private com.google.protobuf.SingleFieldBuilder<
org.apache.drill.exec.proto.CoordinationProtos.Roles, org.apache.drill.exec.proto.CoordinationProtos.Roles.Builder, org.apache.drill.exec.proto.CoordinationProtos.RolesOrBuilder> rolesBuilder_;
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public boolean hasRoles() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
+ return ((bitField0_ & 0x00000010) == 0x00000010);
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public org.apache.drill.exec.proto.CoordinationProtos.Roles getRoles() {
if (rolesBuilder_ == null) {
@@ -743,7 +824,7 @@ public final class CoordinationProtos {
}
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public Builder setRoles(org.apache.drill.exec.proto.CoordinationProtos.Roles value) {
if (rolesBuilder_ == null) {
@@ -755,11 +836,11 @@ public final class CoordinationProtos {
} else {
rolesBuilder_.setMessage(value);
}
- bitField0_ |= 0x00000008;
+ bitField0_ |= 0x00000010;
return this;
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public Builder setRoles(
org.apache.drill.exec.proto.CoordinationProtos.Roles.Builder builderForValue) {
@@ -769,15 +850,15 @@ public final class CoordinationProtos {
} else {
rolesBuilder_.setMessage(builderForValue.build());
}
- bitField0_ |= 0x00000008;
+ bitField0_ |= 0x00000010;
return this;
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public Builder mergeRoles(org.apache.drill.exec.proto.CoordinationProtos.Roles value) {
if (rolesBuilder_ == null) {
- if (((bitField0_ & 0x00000008) == 0x00000008) &&
+ if (((bitField0_ & 0x00000010) == 0x00000010) &&
roles_ != org.apache.drill.exec.proto.CoordinationProtos.Roles.getDefaultInstance()) {
roles_ =
org.apache.drill.exec.proto.CoordinationProtos.Roles.newBuilder(roles_).mergeFrom(value).buildPartial();
@@ -788,11 +869,11 @@ public final class CoordinationProtos {
} else {
rolesBuilder_.mergeFrom(value);
}
- bitField0_ |= 0x00000008;
+ bitField0_ |= 0x00000010;
return this;
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public Builder clearRoles() {
if (rolesBuilder_ == null) {
@@ -801,19 +882,19 @@ public final class CoordinationProtos {
} else {
rolesBuilder_.clear();
}
- bitField0_ = (bitField0_ & ~0x00000008);
+ bitField0_ = (bitField0_ & ~0x00000010);
return this;
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public org.apache.drill.exec.proto.CoordinationProtos.Roles.Builder getRolesBuilder() {
- bitField0_ |= 0x00000008;
+ bitField0_ |= 0x00000010;
onChanged();
return getRolesFieldBuilder().getBuilder();
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
public org.apache.drill.exec.proto.CoordinationProtos.RolesOrBuilder getRolesOrBuilder() {
if (rolesBuilder_ != null) {
@@ -823,7 +904,7 @@ public final class CoordinationProtos {
}
}
/**
- * <code>optional .exec.Roles roles = 4;</code>
+ * <code>optional .exec.Roles roles = 5;</code>
*/
private com.google.protobuf.SingleFieldBuilder<
org.apache.drill.exec.proto.CoordinationProtos.Roles, org.apache.drill.exec.proto.CoordinationProtos.Roles.Builder, org.apache.drill.exec.proto.CoordinationProtos.RolesOrBuilder>
@@ -2338,18 +2419,18 @@ public final class CoordinationProtos {
descriptor;
static {
java.lang.String[] descriptorData = {
- "\n\022Coordination.proto\022\004exec\"d\n\020DrillbitEn" +
+ "\n\022Coordination.proto\022\004exec\"{\n\020DrillbitEn" +
"dpoint\022\017\n\007address\030\001 \001(\t\022\021\n\tuser_port\030\002 \001" +
- "(\005\022\020\n\010bit_port\030\003 \001(\005\022\032\n\005roles\030\004 \001(\0132\013.ex" +
- "ec.Roles\"i\n\024DrillServiceInstance\022\n\n\002id\030\001" +
- " \001(\t\022\033\n\023registrationTimeUTC\030\002 \001(\003\022(\n\010end" +
- "point\030\003 \001(\0132\026.exec.DrillbitEndpoint\"\227\001\n\005" +
- "Roles\022\027\n\tsql_query\030\001 \001(\010:\004true\022\032\n\014logica" +
- "l_plan\030\002 \001(\010:\004true\022\033\n\rphysical_plan\030\003 \001(" +
- "\010:\004true\022\033\n\rjava_executor\030\004 \001(\010:\004true\022\037\n\021" +
- "distributed_cache\030\005 \001(\010:\004trueB3\n\033org.apa",
- "che.drill.exec.protoB\022CoordinationProtos" +
- "H\001"
+ "(\005\022\024\n\014control_port\030\003 \001(\005\022\021\n\tdata_port\030\004 " +
+ "\001(\005\022\032\n\005roles\030\005 \001(\0132\013.exec.Roles\"i\n\024Drill" +
+ "ServiceInstance\022\n\n\002id\030\001 \001(\t\022\033\n\023registrat" +
+ "ionTimeUTC\030\002 \001(\003\022(\n\010endpoint\030\003 \001(\0132\026.exe" +
+ "c.DrillbitEndpoint\"\227\001\n\005Roles\022\027\n\tsql_quer" +
+ "y\030\001 \001(\010:\004true\022\032\n\014logical_plan\030\002 \001(\010:\004tru" +
+ "e\022\033\n\rphysical_plan\030\003 \001(\010:\004true\022\033\n\rjava_e" +
+ "xecutor\030\004 \001(\010:\004true\022\037\n\021distributed_cache",
+ "\030\005 \001(\010:\004trueB3\n\033org.apache.drill.exec.pr" +
+ "otoB\022CoordinationProtosH\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -2361,7 +2442,7 @@ public final class CoordinationProtos {
internal_static_exec_DrillbitEndpoint_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_exec_DrillbitEndpoint_descriptor,
- new java.lang.String[] { "Address", "UserPort", "BitPort", "Roles", });
+ new java.lang.String[] { "Address", "UserPort", "ControlPort", "DataPort", "Roles", });
internal_static_exec_DrillServiceInstance_descriptor =
getDescriptor().getMessageTypes().get(1);
internal_static_exec_DrillServiceInstance_fieldAccessorTable = new
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/ExecProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/ExecProtos.java
index 2ec268368..7ca17f1e3 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/ExecProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/ExecProtos.java
@@ -25,5579 +25,6 @@ public final class ExecProtos {
public static void registerAllExtensions(
com.google.protobuf.ExtensionRegistry registry) {
}
- /**
- * Protobuf enum {@code exec.bit.RpcType}
- *
- * <pre>
- *&#47;/// UserToBit RPC ///////
- * </pre>
- */
- public enum RpcType
- implements com.google.protobuf.ProtocolMessageEnum {
- /**
- * <code>HANDSHAKE = 0;</code>
- */
- HANDSHAKE(0, 0),
- /**
- * <code>ACK = 1;</code>
- */
- ACK(1, 1),
- /**
- * <code>GOODBYE = 2;</code>
- */
- GOODBYE(2, 2),
- /**
- * <code>REQ_INIATILIZE_FRAGMENT = 3;</code>
- *
- * <pre>
- * bit requests
- * </pre>
- */
- REQ_INIATILIZE_FRAGMENT(3, 3),
- /**
- * <code>REQ_RECORD_BATCH = 4;</code>
- *
- * <pre>
- * send record batch overview, returns Ack
- * </pre>
- */
- REQ_RECORD_BATCH(4, 4),
- /**
- * <code>REQ_CANCEL_FRAGMENT = 6;</code>
- *
- * <pre>
- * send a cancellation message for a fragment, returns Ack
- * </pre>
- */
- REQ_CANCEL_FRAGMENT(5, 6),
- /**
- * <code>REQ_FRAGMENT_STATUS = 7;</code>
- *
- * <pre>
- * get a fragment status, returns FragmentStatus
- * </pre>
- */
- REQ_FRAGMENT_STATUS(6, 7),
- /**
- * <code>REQ_BIT_STATUS = 8;</code>
- *
- * <pre>
- * get bit status.
- * </pre>
- */
- REQ_BIT_STATUS(7, 8),
- /**
- * <code>RESP_FRAGMENT_HANDLE = 9;</code>
- *
- * <pre>
- * bit responses
- * </pre>
- */
- RESP_FRAGMENT_HANDLE(8, 9),
- /**
- * <code>RESP_FRAGMENT_STATUS = 10;</code>
- */
- RESP_FRAGMENT_STATUS(9, 10),
- /**
- * <code>RESP_BIT_STATUS = 11;</code>
- */
- RESP_BIT_STATUS(10, 11),
- ;
-
- /**
- * <code>HANDSHAKE = 0;</code>
- */
- public static final int HANDSHAKE_VALUE = 0;
- /**
- * <code>ACK = 1;</code>
- */
- public static final int ACK_VALUE = 1;
- /**
- * <code>GOODBYE = 2;</code>
- */
- public static final int GOODBYE_VALUE = 2;
- /**
- * <code>REQ_INIATILIZE_FRAGMENT = 3;</code>
- *
- * <pre>
- * bit requests
- * </pre>
- */
- public static final int REQ_INIATILIZE_FRAGMENT_VALUE = 3;
- /**
- * <code>REQ_RECORD_BATCH = 4;</code>
- *
- * <pre>
- * send record batch overview, returns Ack
- * </pre>
- */
- public static final int REQ_RECORD_BATCH_VALUE = 4;
- /**
- * <code>REQ_CANCEL_FRAGMENT = 6;</code>
- *
- * <pre>
- * send a cancellation message for a fragment, returns Ack
- * </pre>
- */
- public static final int REQ_CANCEL_FRAGMENT_VALUE = 6;
- /**
- * <code>REQ_FRAGMENT_STATUS = 7;</code>
- *
- * <pre>
- * get a fragment status, returns FragmentStatus
- * </pre>
- */
- public static final int REQ_FRAGMENT_STATUS_VALUE = 7;
- /**
- * <code>REQ_BIT_STATUS = 8;</code>
- *
- * <pre>
- * get bit status.
- * </pre>
- */
- public static final int REQ_BIT_STATUS_VALUE = 8;
- /**
- * <code>RESP_FRAGMENT_HANDLE = 9;</code>
- *
- * <pre>
- * bit responses
- * </pre>
- */
- public static final int RESP_FRAGMENT_HANDLE_VALUE = 9;
- /**
- * <code>RESP_FRAGMENT_STATUS = 10;</code>
- */
- public static final int RESP_FRAGMENT_STATUS_VALUE = 10;
- /**
- * <code>RESP_BIT_STATUS = 11;</code>
- */
- public static final int RESP_BIT_STATUS_VALUE = 11;
-
-
- public final int getNumber() { return value; }
-
- public static RpcType valueOf(int value) {
- switch (value) {
- case 0: return HANDSHAKE;
- case 1: return ACK;
- case 2: return GOODBYE;
- case 3: return REQ_INIATILIZE_FRAGMENT;
- case 4: return REQ_RECORD_BATCH;
- case 6: return REQ_CANCEL_FRAGMENT;
- case 7: return REQ_FRAGMENT_STATUS;
- case 8: return REQ_BIT_STATUS;
- case 9: return RESP_FRAGMENT_HANDLE;
- case 10: return RESP_FRAGMENT_STATUS;
- case 11: return RESP_BIT_STATUS;
- default: return null;
- }
- }
-
- public static com.google.protobuf.Internal.EnumLiteMap<RpcType>
- internalGetValueMap() {
- return internalValueMap;
- }
- private static com.google.protobuf.Internal.EnumLiteMap<RpcType>
- internalValueMap =
- new com.google.protobuf.Internal.EnumLiteMap<RpcType>() {
- public RpcType findValueByNumber(int number) {
- return RpcType.valueOf(number);
- }
- };
-
- public final com.google.protobuf.Descriptors.EnumValueDescriptor
- getValueDescriptor() {
- return getDescriptor().getValues().get(index);
- }
- public final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptorForType() {
- return getDescriptor();
- }
- public static final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.getDescriptor().getEnumTypes().get(0);
- }
-
- private static final RpcType[] VALUES = values();
-
- public static RpcType valueOf(
- com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
- if (desc.getType() != getDescriptor()) {
- throw new java.lang.IllegalArgumentException(
- "EnumValueDescriptor is not for this type.");
- }
- return VALUES[desc.getIndex()];
- }
-
- private final int index;
- private final int value;
-
- private RpcType(int index, int value) {
- this.index = index;
- this.value = value;
- }
-
- // @@protoc_insertion_point(enum_scope:exec.bit.RpcType)
- }
-
- public interface BitHandshakeOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
-
- // optional int32 rpc_version = 1;
- /**
- * <code>optional int32 rpc_version = 1;</code>
- */
- boolean hasRpcVersion();
- /**
- * <code>optional int32 rpc_version = 1;</code>
- */
- int getRpcVersion();
-
- // optional .exec.DrillbitEndpoint endpoint = 2;
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- boolean hasEndpoint();
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint();
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder();
- }
- /**
- * Protobuf type {@code exec.bit.BitHandshake}
- */
- public static final class BitHandshake extends
- com.google.protobuf.GeneratedMessage
- implements BitHandshakeOrBuilder {
- // Use BitHandshake.newBuilder() to construct.
- private BitHandshake(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
- super(builder);
- this.unknownFields = builder.getUnknownFields();
- }
- private BitHandshake(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
- private static final BitHandshake defaultInstance;
- public static BitHandshake getDefaultInstance() {
- return defaultInstance;
- }
-
- public BitHandshake getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- private final com.google.protobuf.UnknownFieldSet unknownFields;
- @java.lang.Override
- public final com.google.protobuf.UnknownFieldSet
- getUnknownFields() {
- return this.unknownFields;
- }
- private BitHandshake(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- initFields();
- int mutable_bitField0_ = 0;
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder();
- try {
- boolean done = false;
- while (!done) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- done = true;
- break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- done = true;
- }
- break;
- }
- case 8: {
- bitField0_ |= 0x00000001;
- rpcVersion_ = input.readInt32();
- break;
- }
- case 18: {
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- subBuilder = endpoint_.toBuilder();
- }
- endpoint_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(endpoint_);
- endpoint_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000002;
- break;
- }
- }
- }
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- throw e.setUnfinishedMessage(this);
- } catch (java.io.IOException e) {
- throw new com.google.protobuf.InvalidProtocolBufferException(
- e.getMessage()).setUnfinishedMessage(this);
- } finally {
- this.unknownFields = unknownFields.build();
- makeExtensionsImmutable();
- }
- }
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitHandshake_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitHandshake_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.BitHandshake.class, org.apache.drill.exec.proto.ExecProtos.BitHandshake.Builder.class);
- }
-
- public static com.google.protobuf.Parser<BitHandshake> PARSER =
- new com.google.protobuf.AbstractParser<BitHandshake>() {
- public BitHandshake parsePartialFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return new BitHandshake(input, extensionRegistry);
- }
- };
-
- @java.lang.Override
- public com.google.protobuf.Parser<BitHandshake> getParserForType() {
- return PARSER;
- }
-
- private int bitField0_;
- // optional int32 rpc_version = 1;
- public static final int RPC_VERSION_FIELD_NUMBER = 1;
- private int rpcVersion_;
- /**
- * <code>optional int32 rpc_version = 1;</code>
- */
- public boolean hasRpcVersion() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional int32 rpc_version = 1;</code>
- */
- public int getRpcVersion() {
- return rpcVersion_;
- }
-
- // optional .exec.DrillbitEndpoint endpoint = 2;
- public static final int ENDPOINT_FIELD_NUMBER = 2;
- private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_;
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public boolean hasEndpoint() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
- return endpoint_;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
- return endpoint_;
- }
-
- private void initFields() {
- rpcVersion_ = 0;
- endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- }
- private byte memoizedIsInitialized = -1;
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) return isInitialized == 1;
-
- memoizedIsInitialized = 1;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeInt32(1, rpcVersion_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- output.writeMessage(2, endpoint_);
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(1, rpcVersion_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(2, endpoint_);
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @java.lang.Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
- }
-
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitHandshake parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.drill.exec.proto.ExecProtos.BitHandshake prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- @java.lang.Override
- protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
- }
- /**
- * Protobuf type {@code exec.bit.BitHandshake}
- */
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.drill.exec.proto.ExecProtos.BitHandshakeOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitHandshake_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitHandshake_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.BitHandshake.class, org.apache.drill.exec.proto.ExecProtos.BitHandshake.Builder.class);
- }
-
- // Construct using org.apache.drill.exec.proto.ExecProtos.BitHandshake.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- getEndpointFieldBuilder();
- }
- }
- private static Builder create() {
- return new Builder();
- }
-
- public Builder clear() {
- super.clear();
- rpcVersion_ = 0;
- bitField0_ = (bitField0_ & ~0x00000001);
- if (endpointBuilder_ == null) {
- endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- } else {
- endpointBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000002);
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(buildPartial());
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitHandshake_descriptor;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.BitHandshake getDefaultInstanceForType() {
- return org.apache.drill.exec.proto.ExecProtos.BitHandshake.getDefaultInstance();
- }
-
- public org.apache.drill.exec.proto.ExecProtos.BitHandshake build() {
- org.apache.drill.exec.proto.ExecProtos.BitHandshake result = buildPartial();
- if (!result.isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return result;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.BitHandshake buildPartial() {
- org.apache.drill.exec.proto.ExecProtos.BitHandshake result = new org.apache.drill.exec.proto.ExecProtos.BitHandshake(this);
- int from_bitField0_ = bitField0_;
- int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
- to_bitField0_ |= 0x00000001;
- }
- result.rpcVersion_ = rpcVersion_;
- if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
- to_bitField0_ |= 0x00000002;
- }
- if (endpointBuilder_ == null) {
- result.endpoint_ = endpoint_;
- } else {
- result.endpoint_ = endpointBuilder_.build();
- }
- result.bitField0_ = to_bitField0_;
- onBuilt();
- return result;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.drill.exec.proto.ExecProtos.BitHandshake) {
- return mergeFrom((org.apache.drill.exec.proto.ExecProtos.BitHandshake)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(org.apache.drill.exec.proto.ExecProtos.BitHandshake other) {
- if (other == org.apache.drill.exec.proto.ExecProtos.BitHandshake.getDefaultInstance()) return this;
- if (other.hasRpcVersion()) {
- setRpcVersion(other.getRpcVersion());
- }
- if (other.hasEndpoint()) {
- mergeEndpoint(other.getEndpoint());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public final boolean isInitialized() {
- return true;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- org.apache.drill.exec.proto.ExecProtos.BitHandshake parsedMessage = null;
- try {
- parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.drill.exec.proto.ExecProtos.BitHandshake) e.getUnfinishedMessage();
- throw e;
- } finally {
- if (parsedMessage != null) {
- mergeFrom(parsedMessage);
- }
- }
- return this;
- }
- private int bitField0_;
-
- // optional int32 rpc_version = 1;
- private int rpcVersion_ ;
- /**
- * <code>optional int32 rpc_version = 1;</code>
- */
- public boolean hasRpcVersion() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional int32 rpc_version = 1;</code>
- */
- public int getRpcVersion() {
- return rpcVersion_;
- }
- /**
- * <code>optional int32 rpc_version = 1;</code>
- */
- public Builder setRpcVersion(int value) {
- bitField0_ |= 0x00000001;
- rpcVersion_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int32 rpc_version = 1;</code>
- */
- public Builder clearRpcVersion() {
- bitField0_ = (bitField0_ & ~0x00000001);
- rpcVersion_ = 0;
- onChanged();
- return this;
- }
-
- // optional .exec.DrillbitEndpoint endpoint = 2;
- private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> endpointBuilder_;
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public boolean hasEndpoint() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
- if (endpointBuilder_ == null) {
- return endpoint_;
- } else {
- return endpointBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public Builder setEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
- if (endpointBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- endpoint_ = value;
- onChanged();
- } else {
- endpointBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000002;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public Builder setEndpoint(
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder builderForValue) {
- if (endpointBuilder_ == null) {
- endpoint_ = builderForValue.build();
- onChanged();
- } else {
- endpointBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000002;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public Builder mergeEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
- if (endpointBuilder_ == null) {
- if (((bitField0_ & 0x00000002) == 0x00000002) &&
- endpoint_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
- endpoint_ =
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(endpoint_).mergeFrom(value).buildPartial();
- } else {
- endpoint_ = value;
- }
- onChanged();
- } else {
- endpointBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000002;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public Builder clearEndpoint() {
- if (endpointBuilder_ == null) {
- endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- onChanged();
- } else {
- endpointBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000002);
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getEndpointBuilder() {
- bitField0_ |= 0x00000002;
- onChanged();
- return getEndpointFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
- if (endpointBuilder_ != null) {
- return endpointBuilder_.getMessageOrBuilder();
- } else {
- return endpoint_;
- }
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 2;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>
- getEndpointFieldBuilder() {
- if (endpointBuilder_ == null) {
- endpointBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>(
- endpoint_,
- getParentForChildren(),
- isClean());
- endpoint_ = null;
- }
- return endpointBuilder_;
- }
-
- // @@protoc_insertion_point(builder_scope:exec.bit.BitHandshake)
- }
-
- static {
- defaultInstance = new BitHandshake(true);
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:exec.bit.BitHandshake)
- }
-
- public interface BitBatchChunkOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
- }
- /**
- * Protobuf type {@code exec.bit.BitBatchChunk}
- */
- public static final class BitBatchChunk extends
- com.google.protobuf.GeneratedMessage
- implements BitBatchChunkOrBuilder {
- // Use BitBatchChunk.newBuilder() to construct.
- private BitBatchChunk(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
- super(builder);
- this.unknownFields = builder.getUnknownFields();
- }
- private BitBatchChunk(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
- private static final BitBatchChunk defaultInstance;
- public static BitBatchChunk getDefaultInstance() {
- return defaultInstance;
- }
-
- public BitBatchChunk getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- private final com.google.protobuf.UnknownFieldSet unknownFields;
- @java.lang.Override
- public final com.google.protobuf.UnknownFieldSet
- getUnknownFields() {
- return this.unknownFields;
- }
- private BitBatchChunk(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- initFields();
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder();
- try {
- boolean done = false;
- while (!done) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- done = true;
- break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- done = true;
- }
- break;
- }
- }
- }
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- throw e.setUnfinishedMessage(this);
- } catch (java.io.IOException e) {
- throw new com.google.protobuf.InvalidProtocolBufferException(
- e.getMessage()).setUnfinishedMessage(this);
- } finally {
- this.unknownFields = unknownFields.build();
- makeExtensionsImmutable();
- }
- }
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitBatchChunk_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitBatchChunk_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.BitBatchChunk.class, org.apache.drill.exec.proto.ExecProtos.BitBatchChunk.Builder.class);
- }
-
- public static com.google.protobuf.Parser<BitBatchChunk> PARSER =
- new com.google.protobuf.AbstractParser<BitBatchChunk>() {
- public BitBatchChunk parsePartialFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return new BitBatchChunk(input, extensionRegistry);
- }
- };
-
- @java.lang.Override
- public com.google.protobuf.Parser<BitBatchChunk> getParserForType() {
- return PARSER;
- }
-
- private void initFields() {
- }
- private byte memoizedIsInitialized = -1;
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) return isInitialized == 1;
-
- memoizedIsInitialized = 1;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @java.lang.Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
- }
-
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.drill.exec.proto.ExecProtos.BitBatchChunk prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- @java.lang.Override
- protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
- }
- /**
- * Protobuf type {@code exec.bit.BitBatchChunk}
- */
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.drill.exec.proto.ExecProtos.BitBatchChunkOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitBatchChunk_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitBatchChunk_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.BitBatchChunk.class, org.apache.drill.exec.proto.ExecProtos.BitBatchChunk.Builder.class);
- }
-
- // Construct using org.apache.drill.exec.proto.ExecProtos.BitBatchChunk.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- }
- }
- private static Builder create() {
- return new Builder();
- }
-
- public Builder clear() {
- super.clear();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(buildPartial());
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitBatchChunk_descriptor;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.BitBatchChunk getDefaultInstanceForType() {
- return org.apache.drill.exec.proto.ExecProtos.BitBatchChunk.getDefaultInstance();
- }
-
- public org.apache.drill.exec.proto.ExecProtos.BitBatchChunk build() {
- org.apache.drill.exec.proto.ExecProtos.BitBatchChunk result = buildPartial();
- if (!result.isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return result;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.BitBatchChunk buildPartial() {
- org.apache.drill.exec.proto.ExecProtos.BitBatchChunk result = new org.apache.drill.exec.proto.ExecProtos.BitBatchChunk(this);
- onBuilt();
- return result;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.drill.exec.proto.ExecProtos.BitBatchChunk) {
- return mergeFrom((org.apache.drill.exec.proto.ExecProtos.BitBatchChunk)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(org.apache.drill.exec.proto.ExecProtos.BitBatchChunk other) {
- if (other == org.apache.drill.exec.proto.ExecProtos.BitBatchChunk.getDefaultInstance()) return this;
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public final boolean isInitialized() {
- return true;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- org.apache.drill.exec.proto.ExecProtos.BitBatchChunk parsedMessage = null;
- try {
- parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.drill.exec.proto.ExecProtos.BitBatchChunk) e.getUnfinishedMessage();
- throw e;
- } finally {
- if (parsedMessage != null) {
- mergeFrom(parsedMessage);
- }
- }
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:exec.bit.BitBatchChunk)
- }
-
- static {
- defaultInstance = new BitBatchChunk(true);
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:exec.bit.BitBatchChunk)
- }
-
- public interface BitStatusOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
-
- // repeated .exec.bit.FragmentStatus fragment_status = 1;
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- java.util.List<org.apache.drill.exec.proto.ExecProtos.FragmentStatus>
- getFragmentStatusList();
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus getFragmentStatus(int index);
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- int getFragmentStatusCount();
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- java.util.List<? extends org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder>
- getFragmentStatusOrBuilderList();
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder getFragmentStatusOrBuilder(
- int index);
- }
- /**
- * Protobuf type {@code exec.bit.BitStatus}
- */
- public static final class BitStatus extends
- com.google.protobuf.GeneratedMessage
- implements BitStatusOrBuilder {
- // Use BitStatus.newBuilder() to construct.
- private BitStatus(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
- super(builder);
- this.unknownFields = builder.getUnknownFields();
- }
- private BitStatus(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
- private static final BitStatus defaultInstance;
- public static BitStatus getDefaultInstance() {
- return defaultInstance;
- }
-
- public BitStatus getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- private final com.google.protobuf.UnknownFieldSet unknownFields;
- @java.lang.Override
- public final com.google.protobuf.UnknownFieldSet
- getUnknownFields() {
- return this.unknownFields;
- }
- private BitStatus(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- initFields();
- int mutable_bitField0_ = 0;
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder();
- try {
- boolean done = false;
- while (!done) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- done = true;
- break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- done = true;
- }
- break;
- }
- case 10: {
- if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
- fragmentStatus_ = new java.util.ArrayList<org.apache.drill.exec.proto.ExecProtos.FragmentStatus>();
- mutable_bitField0_ |= 0x00000001;
- }
- fragmentStatus_.add(input.readMessage(org.apache.drill.exec.proto.ExecProtos.FragmentStatus.PARSER, extensionRegistry));
- break;
- }
- }
- }
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- throw e.setUnfinishedMessage(this);
- } catch (java.io.IOException e) {
- throw new com.google.protobuf.InvalidProtocolBufferException(
- e.getMessage()).setUnfinishedMessage(this);
- } finally {
- if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
- fragmentStatus_ = java.util.Collections.unmodifiableList(fragmentStatus_);
- }
- this.unknownFields = unknownFields.build();
- makeExtensionsImmutable();
- }
- }
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitStatus_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitStatus_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.BitStatus.class, org.apache.drill.exec.proto.ExecProtos.BitStatus.Builder.class);
- }
-
- public static com.google.protobuf.Parser<BitStatus> PARSER =
- new com.google.protobuf.AbstractParser<BitStatus>() {
- public BitStatus parsePartialFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return new BitStatus(input, extensionRegistry);
- }
- };
-
- @java.lang.Override
- public com.google.protobuf.Parser<BitStatus> getParserForType() {
- return PARSER;
- }
-
- // repeated .exec.bit.FragmentStatus fragment_status = 1;
- public static final int FRAGMENT_STATUS_FIELD_NUMBER = 1;
- private java.util.List<org.apache.drill.exec.proto.ExecProtos.FragmentStatus> fragmentStatus_;
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public java.util.List<org.apache.drill.exec.proto.ExecProtos.FragmentStatus> getFragmentStatusList() {
- return fragmentStatus_;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public java.util.List<? extends org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder>
- getFragmentStatusOrBuilderList() {
- return fragmentStatus_;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public int getFragmentStatusCount() {
- return fragmentStatus_.size();
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus getFragmentStatus(int index) {
- return fragmentStatus_.get(index);
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder getFragmentStatusOrBuilder(
- int index) {
- return fragmentStatus_.get(index);
- }
-
- private void initFields() {
- fragmentStatus_ = java.util.Collections.emptyList();
- }
- private byte memoizedIsInitialized = -1;
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) return isInitialized == 1;
-
- memoizedIsInitialized = 1;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- for (int i = 0; i < fragmentStatus_.size(); i++) {
- output.writeMessage(1, fragmentStatus_.get(i));
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- for (int i = 0; i < fragmentStatus_.size(); i++) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(1, fragmentStatus_.get(i));
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @java.lang.Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
- }
-
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.BitStatus parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.drill.exec.proto.ExecProtos.BitStatus prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- @java.lang.Override
- protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
- }
- /**
- * Protobuf type {@code exec.bit.BitStatus}
- */
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.drill.exec.proto.ExecProtos.BitStatusOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitStatus_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitStatus_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.BitStatus.class, org.apache.drill.exec.proto.ExecProtos.BitStatus.Builder.class);
- }
-
- // Construct using org.apache.drill.exec.proto.ExecProtos.BitStatus.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- getFragmentStatusFieldBuilder();
- }
- }
- private static Builder create() {
- return new Builder();
- }
-
- public Builder clear() {
- super.clear();
- if (fragmentStatusBuilder_ == null) {
- fragmentStatus_ = java.util.Collections.emptyList();
- bitField0_ = (bitField0_ & ~0x00000001);
- } else {
- fragmentStatusBuilder_.clear();
- }
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(buildPartial());
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_BitStatus_descriptor;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.BitStatus getDefaultInstanceForType() {
- return org.apache.drill.exec.proto.ExecProtos.BitStatus.getDefaultInstance();
- }
-
- public org.apache.drill.exec.proto.ExecProtos.BitStatus build() {
- org.apache.drill.exec.proto.ExecProtos.BitStatus result = buildPartial();
- if (!result.isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return result;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.BitStatus buildPartial() {
- org.apache.drill.exec.proto.ExecProtos.BitStatus result = new org.apache.drill.exec.proto.ExecProtos.BitStatus(this);
- int from_bitField0_ = bitField0_;
- if (fragmentStatusBuilder_ == null) {
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- fragmentStatus_ = java.util.Collections.unmodifiableList(fragmentStatus_);
- bitField0_ = (bitField0_ & ~0x00000001);
- }
- result.fragmentStatus_ = fragmentStatus_;
- } else {
- result.fragmentStatus_ = fragmentStatusBuilder_.build();
- }
- onBuilt();
- return result;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.drill.exec.proto.ExecProtos.BitStatus) {
- return mergeFrom((org.apache.drill.exec.proto.ExecProtos.BitStatus)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(org.apache.drill.exec.proto.ExecProtos.BitStatus other) {
- if (other == org.apache.drill.exec.proto.ExecProtos.BitStatus.getDefaultInstance()) return this;
- if (fragmentStatusBuilder_ == null) {
- if (!other.fragmentStatus_.isEmpty()) {
- if (fragmentStatus_.isEmpty()) {
- fragmentStatus_ = other.fragmentStatus_;
- bitField0_ = (bitField0_ & ~0x00000001);
- } else {
- ensureFragmentStatusIsMutable();
- fragmentStatus_.addAll(other.fragmentStatus_);
- }
- onChanged();
- }
- } else {
- if (!other.fragmentStatus_.isEmpty()) {
- if (fragmentStatusBuilder_.isEmpty()) {
- fragmentStatusBuilder_.dispose();
- fragmentStatusBuilder_ = null;
- fragmentStatus_ = other.fragmentStatus_;
- bitField0_ = (bitField0_ & ~0x00000001);
- fragmentStatusBuilder_ =
- com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
- getFragmentStatusFieldBuilder() : null;
- } else {
- fragmentStatusBuilder_.addAllMessages(other.fragmentStatus_);
- }
- }
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public final boolean isInitialized() {
- return true;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- org.apache.drill.exec.proto.ExecProtos.BitStatus parsedMessage = null;
- try {
- parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.drill.exec.proto.ExecProtos.BitStatus) e.getUnfinishedMessage();
- throw e;
- } finally {
- if (parsedMessage != null) {
- mergeFrom(parsedMessage);
- }
- }
- return this;
- }
- private int bitField0_;
-
- // repeated .exec.bit.FragmentStatus fragment_status = 1;
- private java.util.List<org.apache.drill.exec.proto.ExecProtos.FragmentStatus> fragmentStatus_ =
- java.util.Collections.emptyList();
- private void ensureFragmentStatusIsMutable() {
- if (!((bitField0_ & 0x00000001) == 0x00000001)) {
- fragmentStatus_ = new java.util.ArrayList<org.apache.drill.exec.proto.ExecProtos.FragmentStatus>(fragmentStatus_);
- bitField0_ |= 0x00000001;
- }
- }
-
- private com.google.protobuf.RepeatedFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus, org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder> fragmentStatusBuilder_;
-
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public java.util.List<org.apache.drill.exec.proto.ExecProtos.FragmentStatus> getFragmentStatusList() {
- if (fragmentStatusBuilder_ == null) {
- return java.util.Collections.unmodifiableList(fragmentStatus_);
- } else {
- return fragmentStatusBuilder_.getMessageList();
- }
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public int getFragmentStatusCount() {
- if (fragmentStatusBuilder_ == null) {
- return fragmentStatus_.size();
- } else {
- return fragmentStatusBuilder_.getCount();
- }
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus getFragmentStatus(int index) {
- if (fragmentStatusBuilder_ == null) {
- return fragmentStatus_.get(index);
- } else {
- return fragmentStatusBuilder_.getMessage(index);
- }
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public Builder setFragmentStatus(
- int index, org.apache.drill.exec.proto.ExecProtos.FragmentStatus value) {
- if (fragmentStatusBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- ensureFragmentStatusIsMutable();
- fragmentStatus_.set(index, value);
- onChanged();
- } else {
- fragmentStatusBuilder_.setMessage(index, value);
- }
- return this;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public Builder setFragmentStatus(
- int index, org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder builderForValue) {
- if (fragmentStatusBuilder_ == null) {
- ensureFragmentStatusIsMutable();
- fragmentStatus_.set(index, builderForValue.build());
- onChanged();
- } else {
- fragmentStatusBuilder_.setMessage(index, builderForValue.build());
- }
- return this;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public Builder addFragmentStatus(org.apache.drill.exec.proto.ExecProtos.FragmentStatus value) {
- if (fragmentStatusBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- ensureFragmentStatusIsMutable();
- fragmentStatus_.add(value);
- onChanged();
- } else {
- fragmentStatusBuilder_.addMessage(value);
- }
- return this;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public Builder addFragmentStatus(
- int index, org.apache.drill.exec.proto.ExecProtos.FragmentStatus value) {
- if (fragmentStatusBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- ensureFragmentStatusIsMutable();
- fragmentStatus_.add(index, value);
- onChanged();
- } else {
- fragmentStatusBuilder_.addMessage(index, value);
- }
- return this;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public Builder addFragmentStatus(
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder builderForValue) {
- if (fragmentStatusBuilder_ == null) {
- ensureFragmentStatusIsMutable();
- fragmentStatus_.add(builderForValue.build());
- onChanged();
- } else {
- fragmentStatusBuilder_.addMessage(builderForValue.build());
- }
- return this;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public Builder addFragmentStatus(
- int index, org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder builderForValue) {
- if (fragmentStatusBuilder_ == null) {
- ensureFragmentStatusIsMutable();
- fragmentStatus_.add(index, builderForValue.build());
- onChanged();
- } else {
- fragmentStatusBuilder_.addMessage(index, builderForValue.build());
- }
- return this;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public Builder addAllFragmentStatus(
- java.lang.Iterable<? extends org.apache.drill.exec.proto.ExecProtos.FragmentStatus> values) {
- if (fragmentStatusBuilder_ == null) {
- ensureFragmentStatusIsMutable();
- super.addAll(values, fragmentStatus_);
- onChanged();
- } else {
- fragmentStatusBuilder_.addAllMessages(values);
- }
- return this;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public Builder clearFragmentStatus() {
- if (fragmentStatusBuilder_ == null) {
- fragmentStatus_ = java.util.Collections.emptyList();
- bitField0_ = (bitField0_ & ~0x00000001);
- onChanged();
- } else {
- fragmentStatusBuilder_.clear();
- }
- return this;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public Builder removeFragmentStatus(int index) {
- if (fragmentStatusBuilder_ == null) {
- ensureFragmentStatusIsMutable();
- fragmentStatus_.remove(index);
- onChanged();
- } else {
- fragmentStatusBuilder_.remove(index);
- }
- return this;
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder getFragmentStatusBuilder(
- int index) {
- return getFragmentStatusFieldBuilder().getBuilder(index);
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder getFragmentStatusOrBuilder(
- int index) {
- if (fragmentStatusBuilder_ == null) {
- return fragmentStatus_.get(index); } else {
- return fragmentStatusBuilder_.getMessageOrBuilder(index);
- }
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public java.util.List<? extends org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder>
- getFragmentStatusOrBuilderList() {
- if (fragmentStatusBuilder_ != null) {
- return fragmentStatusBuilder_.getMessageOrBuilderList();
- } else {
- return java.util.Collections.unmodifiableList(fragmentStatus_);
- }
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder addFragmentStatusBuilder() {
- return getFragmentStatusFieldBuilder().addBuilder(
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus.getDefaultInstance());
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder addFragmentStatusBuilder(
- int index) {
- return getFragmentStatusFieldBuilder().addBuilder(
- index, org.apache.drill.exec.proto.ExecProtos.FragmentStatus.getDefaultInstance());
- }
- /**
- * <code>repeated .exec.bit.FragmentStatus fragment_status = 1;</code>
- */
- public java.util.List<org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder>
- getFragmentStatusBuilderList() {
- return getFragmentStatusFieldBuilder().getBuilderList();
- }
- private com.google.protobuf.RepeatedFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus, org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder>
- getFragmentStatusFieldBuilder() {
- if (fragmentStatusBuilder_ == null) {
- fragmentStatusBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus, org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder>(
- fragmentStatus_,
- ((bitField0_ & 0x00000001) == 0x00000001),
- getParentForChildren(),
- isClean());
- fragmentStatus_ = null;
- }
- return fragmentStatusBuilder_;
- }
-
- // @@protoc_insertion_point(builder_scope:exec.bit.BitStatus)
- }
-
- static {
- defaultInstance = new BitStatus(true);
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:exec.bit.BitStatus)
- }
-
- public interface FragmentStatusOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
-
- // optional int64 memory_use = 1;
- /**
- * <code>optional int64 memory_use = 1;</code>
- */
- boolean hasMemoryUse();
- /**
- * <code>optional int64 memory_use = 1;</code>
- */
- long getMemoryUse();
-
- // optional int64 batches_completed = 2;
- /**
- * <code>optional int64 batches_completed = 2;</code>
- */
- boolean hasBatchesCompleted();
- /**
- * <code>optional int64 batches_completed = 2;</code>
- */
- long getBatchesCompleted();
-
- // optional int64 records_completed = 3;
- /**
- * <code>optional int64 records_completed = 3;</code>
- */
- boolean hasRecordsCompleted();
- /**
- * <code>optional int64 records_completed = 3;</code>
- */
- long getRecordsCompleted();
-
- // optional int32 estimated_completion_percentage = 4;
- /**
- * <code>optional int32 estimated_completion_percentage = 4;</code>
- */
- boolean hasEstimatedCompletionPercentage();
- /**
- * <code>optional int32 estimated_completion_percentage = 4;</code>
- */
- int getEstimatedCompletionPercentage();
-
- // optional .exec.bit.FragmentStatus.FragmentState state = 5;
- /**
- * <code>optional .exec.bit.FragmentStatus.FragmentState state = 5;</code>
- */
- boolean hasState();
- /**
- * <code>optional .exec.bit.FragmentStatus.FragmentState state = 5;</code>
- */
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState getState();
-
- // optional int64 data_processed = 6;
- /**
- * <code>optional int64 data_processed = 6;</code>
- */
- boolean hasDataProcessed();
- /**
- * <code>optional int64 data_processed = 6;</code>
- */
- long getDataProcessed();
-
- // optional .exec.bit.FragmentHandle handle = 7;
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- boolean hasHandle();
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle();
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder();
-
- // optional .exec.shared.DrillPBError error = 8;
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- boolean hasError();
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- org.apache.drill.exec.proto.UserBitShared.DrillPBError getError();
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder();
-
- // optional int64 running_time = 9;
- /**
- * <code>optional int64 running_time = 9;</code>
- */
- boolean hasRunningTime();
- /**
- * <code>optional int64 running_time = 9;</code>
- */
- long getRunningTime();
- }
- /**
- * Protobuf type {@code exec.bit.FragmentStatus}
- */
- public static final class FragmentStatus extends
- com.google.protobuf.GeneratedMessage
- implements FragmentStatusOrBuilder {
- // Use FragmentStatus.newBuilder() to construct.
- private FragmentStatus(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
- super(builder);
- this.unknownFields = builder.getUnknownFields();
- }
- private FragmentStatus(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
- private static final FragmentStatus defaultInstance;
- public static FragmentStatus getDefaultInstance() {
- return defaultInstance;
- }
-
- public FragmentStatus getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- private final com.google.protobuf.UnknownFieldSet unknownFields;
- @java.lang.Override
- public final com.google.protobuf.UnknownFieldSet
- getUnknownFields() {
- return this.unknownFields;
- }
- private FragmentStatus(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- initFields();
- int mutable_bitField0_ = 0;
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder();
- try {
- boolean done = false;
- while (!done) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- done = true;
- break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- done = true;
- }
- break;
- }
- case 8: {
- bitField0_ |= 0x00000001;
- memoryUse_ = input.readInt64();
- break;
- }
- case 16: {
- bitField0_ |= 0x00000002;
- batchesCompleted_ = input.readInt64();
- break;
- }
- case 24: {
- bitField0_ |= 0x00000004;
- recordsCompleted_ = input.readInt64();
- break;
- }
- case 32: {
- bitField0_ |= 0x00000008;
- estimatedCompletionPercentage_ = input.readInt32();
- break;
- }
- case 40: {
- int rawValue = input.readEnum();
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState value = org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState.valueOf(rawValue);
- if (value == null) {
- unknownFields.mergeVarintField(5, rawValue);
- } else {
- bitField0_ |= 0x00000010;
- state_ = value;
- }
- break;
- }
- case 48: {
- bitField0_ |= 0x00000020;
- dataProcessed_ = input.readInt64();
- break;
- }
- case 58: {
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder subBuilder = null;
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- subBuilder = handle_.toBuilder();
- }
- handle_ = input.readMessage(org.apache.drill.exec.proto.ExecProtos.FragmentHandle.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(handle_);
- handle_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000040;
- break;
- }
- case 66: {
- org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder subBuilder = null;
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
- subBuilder = error_.toBuilder();
- }
- error_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.DrillPBError.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(error_);
- error_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000080;
- break;
- }
- case 72: {
- bitField0_ |= 0x00000100;
- runningTime_ = input.readInt64();
- break;
- }
- }
- }
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- throw e.setUnfinishedMessage(this);
- } catch (java.io.IOException e) {
- throw new com.google.protobuf.InvalidProtocolBufferException(
- e.getMessage()).setUnfinishedMessage(this);
- } finally {
- this.unknownFields = unknownFields.build();
- makeExtensionsImmutable();
- }
- }
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentStatus_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentStatus_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus.class, org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder.class);
- }
-
- public static com.google.protobuf.Parser<FragmentStatus> PARSER =
- new com.google.protobuf.AbstractParser<FragmentStatus>() {
- public FragmentStatus parsePartialFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return new FragmentStatus(input, extensionRegistry);
- }
- };
-
- @java.lang.Override
- public com.google.protobuf.Parser<FragmentStatus> getParserForType() {
- return PARSER;
- }
-
- /**
- * Protobuf enum {@code exec.bit.FragmentStatus.FragmentState}
- */
- public enum FragmentState
- implements com.google.protobuf.ProtocolMessageEnum {
- /**
- * <code>SENDING = 0;</code>
- */
- SENDING(0, 0),
- /**
- * <code>AWAITING_ALLOCATION = 1;</code>
- */
- AWAITING_ALLOCATION(1, 1),
- /**
- * <code>RUNNING = 2;</code>
- */
- RUNNING(2, 2),
- /**
- * <code>FINISHED = 3;</code>
- */
- FINISHED(3, 3),
- /**
- * <code>CANCELLED = 4;</code>
- */
- CANCELLED(4, 4),
- /**
- * <code>FAILED = 5;</code>
- */
- FAILED(5, 5),
- ;
-
- /**
- * <code>SENDING = 0;</code>
- */
- public static final int SENDING_VALUE = 0;
- /**
- * <code>AWAITING_ALLOCATION = 1;</code>
- */
- public static final int AWAITING_ALLOCATION_VALUE = 1;
- /**
- * <code>RUNNING = 2;</code>
- */
- public static final int RUNNING_VALUE = 2;
- /**
- * <code>FINISHED = 3;</code>
- */
- public static final int FINISHED_VALUE = 3;
- /**
- * <code>CANCELLED = 4;</code>
- */
- public static final int CANCELLED_VALUE = 4;
- /**
- * <code>FAILED = 5;</code>
- */
- public static final int FAILED_VALUE = 5;
-
-
- public final int getNumber() { return value; }
-
- public static FragmentState valueOf(int value) {
- switch (value) {
- case 0: return SENDING;
- case 1: return AWAITING_ALLOCATION;
- case 2: return RUNNING;
- case 3: return FINISHED;
- case 4: return CANCELLED;
- case 5: return FAILED;
- default: return null;
- }
- }
-
- public static com.google.protobuf.Internal.EnumLiteMap<FragmentState>
- internalGetValueMap() {
- return internalValueMap;
- }
- private static com.google.protobuf.Internal.EnumLiteMap<FragmentState>
- internalValueMap =
- new com.google.protobuf.Internal.EnumLiteMap<FragmentState>() {
- public FragmentState findValueByNumber(int number) {
- return FragmentState.valueOf(number);
- }
- };
-
- public final com.google.protobuf.Descriptors.EnumValueDescriptor
- getValueDescriptor() {
- return getDescriptor().getValues().get(index);
- }
- public final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptorForType() {
- return getDescriptor();
- }
- public static final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.FragmentStatus.getDescriptor().getEnumTypes().get(0);
- }
-
- private static final FragmentState[] VALUES = values();
-
- public static FragmentState valueOf(
- com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
- if (desc.getType() != getDescriptor()) {
- throw new java.lang.IllegalArgumentException(
- "EnumValueDescriptor is not for this type.");
- }
- return VALUES[desc.getIndex()];
- }
-
- private final int index;
- private final int value;
-
- private FragmentState(int index, int value) {
- this.index = index;
- this.value = value;
- }
-
- // @@protoc_insertion_point(enum_scope:exec.bit.FragmentStatus.FragmentState)
- }
-
- private int bitField0_;
- // optional int64 memory_use = 1;
- public static final int MEMORY_USE_FIELD_NUMBER = 1;
- private long memoryUse_;
- /**
- * <code>optional int64 memory_use = 1;</code>
- */
- public boolean hasMemoryUse() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional int64 memory_use = 1;</code>
- */
- public long getMemoryUse() {
- return memoryUse_;
- }
-
- // optional int64 batches_completed = 2;
- public static final int BATCHES_COMPLETED_FIELD_NUMBER = 2;
- private long batchesCompleted_;
- /**
- * <code>optional int64 batches_completed = 2;</code>
- */
- public boolean hasBatchesCompleted() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional int64 batches_completed = 2;</code>
- */
- public long getBatchesCompleted() {
- return batchesCompleted_;
- }
-
- // optional int64 records_completed = 3;
- public static final int RECORDS_COMPLETED_FIELD_NUMBER = 3;
- private long recordsCompleted_;
- /**
- * <code>optional int64 records_completed = 3;</code>
- */
- public boolean hasRecordsCompleted() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- /**
- * <code>optional int64 records_completed = 3;</code>
- */
- public long getRecordsCompleted() {
- return recordsCompleted_;
- }
-
- // optional int32 estimated_completion_percentage = 4;
- public static final int ESTIMATED_COMPLETION_PERCENTAGE_FIELD_NUMBER = 4;
- private int estimatedCompletionPercentage_;
- /**
- * <code>optional int32 estimated_completion_percentage = 4;</code>
- */
- public boolean hasEstimatedCompletionPercentage() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
- }
- /**
- * <code>optional int32 estimated_completion_percentage = 4;</code>
- */
- public int getEstimatedCompletionPercentage() {
- return estimatedCompletionPercentage_;
- }
-
- // optional .exec.bit.FragmentStatus.FragmentState state = 5;
- public static final int STATE_FIELD_NUMBER = 5;
- private org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState state_;
- /**
- * <code>optional .exec.bit.FragmentStatus.FragmentState state = 5;</code>
- */
- public boolean hasState() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- /**
- * <code>optional .exec.bit.FragmentStatus.FragmentState state = 5;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState getState() {
- return state_;
- }
-
- // optional int64 data_processed = 6;
- public static final int DATA_PROCESSED_FIELD_NUMBER = 6;
- private long dataProcessed_;
- /**
- * <code>optional int64 data_processed = 6;</code>
- */
- public boolean hasDataProcessed() {
- return ((bitField0_ & 0x00000020) == 0x00000020);
- }
- /**
- * <code>optional int64 data_processed = 6;</code>
- */
- public long getDataProcessed() {
- return dataProcessed_;
- }
-
- // optional .exec.bit.FragmentHandle handle = 7;
- public static final int HANDLE_FIELD_NUMBER = 7;
- private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_;
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public boolean hasHandle() {
- return ((bitField0_ & 0x00000040) == 0x00000040);
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
- return handle_;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
- return handle_;
- }
-
- // optional .exec.shared.DrillPBError error = 8;
- public static final int ERROR_FIELD_NUMBER = 8;
- private org.apache.drill.exec.proto.UserBitShared.DrillPBError error_;
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public boolean hasError() {
- return ((bitField0_ & 0x00000080) == 0x00000080);
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError() {
- return error_;
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder() {
- return error_;
- }
-
- // optional int64 running_time = 9;
- public static final int RUNNING_TIME_FIELD_NUMBER = 9;
- private long runningTime_;
- /**
- * <code>optional int64 running_time = 9;</code>
- */
- public boolean hasRunningTime() {
- return ((bitField0_ & 0x00000100) == 0x00000100);
- }
- /**
- * <code>optional int64 running_time = 9;</code>
- */
- public long getRunningTime() {
- return runningTime_;
- }
-
- private void initFields() {
- memoryUse_ = 0L;
- batchesCompleted_ = 0L;
- recordsCompleted_ = 0L;
- estimatedCompletionPercentage_ = 0;
- state_ = org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState.SENDING;
- dataProcessed_ = 0L;
- handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
- runningTime_ = 0L;
- }
- private byte memoizedIsInitialized = -1;
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) return isInitialized == 1;
-
- memoizedIsInitialized = 1;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeInt64(1, memoryUse_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- output.writeInt64(2, batchesCompleted_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- output.writeInt64(3, recordsCompleted_);
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- output.writeInt32(4, estimatedCompletionPercentage_);
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- output.writeEnum(5, state_.getNumber());
- }
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
- output.writeInt64(6, dataProcessed_);
- }
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- output.writeMessage(7, handle_);
- }
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
- output.writeMessage(8, error_);
- }
- if (((bitField0_ & 0x00000100) == 0x00000100)) {
- output.writeInt64(9, runningTime_);
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt64Size(1, memoryUse_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt64Size(2, batchesCompleted_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt64Size(3, recordsCompleted_);
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(4, estimatedCompletionPercentage_);
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- size += com.google.protobuf.CodedOutputStream
- .computeEnumSize(5, state_.getNumber());
- }
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt64Size(6, dataProcessed_);
- }
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(7, handle_);
- }
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(8, error_);
- }
- if (((bitField0_ & 0x00000100) == 0x00000100)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt64Size(9, runningTime_);
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @java.lang.Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
- }
-
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentStatus parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.drill.exec.proto.ExecProtos.FragmentStatus prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- @java.lang.Override
- protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
- }
- /**
- * Protobuf type {@code exec.bit.FragmentStatus}
- */
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.drill.exec.proto.ExecProtos.FragmentStatusOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentStatus_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentStatus_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus.class, org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder.class);
- }
-
- // Construct using org.apache.drill.exec.proto.ExecProtos.FragmentStatus.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- getHandleFieldBuilder();
- getErrorFieldBuilder();
- }
- }
- private static Builder create() {
- return new Builder();
- }
-
- public Builder clear() {
- super.clear();
- memoryUse_ = 0L;
- bitField0_ = (bitField0_ & ~0x00000001);
- batchesCompleted_ = 0L;
- bitField0_ = (bitField0_ & ~0x00000002);
- recordsCompleted_ = 0L;
- bitField0_ = (bitField0_ & ~0x00000004);
- estimatedCompletionPercentage_ = 0;
- bitField0_ = (bitField0_ & ~0x00000008);
- state_ = org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState.SENDING;
- bitField0_ = (bitField0_ & ~0x00000010);
- dataProcessed_ = 0L;
- bitField0_ = (bitField0_ & ~0x00000020);
- if (handleBuilder_ == null) {
- handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- } else {
- handleBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000040);
- if (errorBuilder_ == null) {
- error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
- } else {
- errorBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000080);
- runningTime_ = 0L;
- bitField0_ = (bitField0_ & ~0x00000100);
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(buildPartial());
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentStatus_descriptor;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus getDefaultInstanceForType() {
- return org.apache.drill.exec.proto.ExecProtos.FragmentStatus.getDefaultInstance();
- }
-
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus build() {
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus result = buildPartial();
- if (!result.isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return result;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus buildPartial() {
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus result = new org.apache.drill.exec.proto.ExecProtos.FragmentStatus(this);
- int from_bitField0_ = bitField0_;
- int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
- to_bitField0_ |= 0x00000001;
- }
- result.memoryUse_ = memoryUse_;
- if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
- to_bitField0_ |= 0x00000002;
- }
- result.batchesCompleted_ = batchesCompleted_;
- if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
- to_bitField0_ |= 0x00000004;
- }
- result.recordsCompleted_ = recordsCompleted_;
- if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
- to_bitField0_ |= 0x00000008;
- }
- result.estimatedCompletionPercentage_ = estimatedCompletionPercentage_;
- if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
- to_bitField0_ |= 0x00000010;
- }
- result.state_ = state_;
- if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
- to_bitField0_ |= 0x00000020;
- }
- result.dataProcessed_ = dataProcessed_;
- if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
- to_bitField0_ |= 0x00000040;
- }
- if (handleBuilder_ == null) {
- result.handle_ = handle_;
- } else {
- result.handle_ = handleBuilder_.build();
- }
- if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
- to_bitField0_ |= 0x00000080;
- }
- if (errorBuilder_ == null) {
- result.error_ = error_;
- } else {
- result.error_ = errorBuilder_.build();
- }
- if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
- to_bitField0_ |= 0x00000100;
- }
- result.runningTime_ = runningTime_;
- result.bitField0_ = to_bitField0_;
- onBuilt();
- return result;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.drill.exec.proto.ExecProtos.FragmentStatus) {
- return mergeFrom((org.apache.drill.exec.proto.ExecProtos.FragmentStatus)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(org.apache.drill.exec.proto.ExecProtos.FragmentStatus other) {
- if (other == org.apache.drill.exec.proto.ExecProtos.FragmentStatus.getDefaultInstance()) return this;
- if (other.hasMemoryUse()) {
- setMemoryUse(other.getMemoryUse());
- }
- if (other.hasBatchesCompleted()) {
- setBatchesCompleted(other.getBatchesCompleted());
- }
- if (other.hasRecordsCompleted()) {
- setRecordsCompleted(other.getRecordsCompleted());
- }
- if (other.hasEstimatedCompletionPercentage()) {
- setEstimatedCompletionPercentage(other.getEstimatedCompletionPercentage());
- }
- if (other.hasState()) {
- setState(other.getState());
- }
- if (other.hasDataProcessed()) {
- setDataProcessed(other.getDataProcessed());
- }
- if (other.hasHandle()) {
- mergeHandle(other.getHandle());
- }
- if (other.hasError()) {
- mergeError(other.getError());
- }
- if (other.hasRunningTime()) {
- setRunningTime(other.getRunningTime());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public final boolean isInitialized() {
- return true;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- org.apache.drill.exec.proto.ExecProtos.FragmentStatus parsedMessage = null;
- try {
- parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.drill.exec.proto.ExecProtos.FragmentStatus) e.getUnfinishedMessage();
- throw e;
- } finally {
- if (parsedMessage != null) {
- mergeFrom(parsedMessage);
- }
- }
- return this;
- }
- private int bitField0_;
-
- // optional int64 memory_use = 1;
- private long memoryUse_ ;
- /**
- * <code>optional int64 memory_use = 1;</code>
- */
- public boolean hasMemoryUse() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional int64 memory_use = 1;</code>
- */
- public long getMemoryUse() {
- return memoryUse_;
- }
- /**
- * <code>optional int64 memory_use = 1;</code>
- */
- public Builder setMemoryUse(long value) {
- bitField0_ |= 0x00000001;
- memoryUse_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int64 memory_use = 1;</code>
- */
- public Builder clearMemoryUse() {
- bitField0_ = (bitField0_ & ~0x00000001);
- memoryUse_ = 0L;
- onChanged();
- return this;
- }
-
- // optional int64 batches_completed = 2;
- private long batchesCompleted_ ;
- /**
- * <code>optional int64 batches_completed = 2;</code>
- */
- public boolean hasBatchesCompleted() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional int64 batches_completed = 2;</code>
- */
- public long getBatchesCompleted() {
- return batchesCompleted_;
- }
- /**
- * <code>optional int64 batches_completed = 2;</code>
- */
- public Builder setBatchesCompleted(long value) {
- bitField0_ |= 0x00000002;
- batchesCompleted_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int64 batches_completed = 2;</code>
- */
- public Builder clearBatchesCompleted() {
- bitField0_ = (bitField0_ & ~0x00000002);
- batchesCompleted_ = 0L;
- onChanged();
- return this;
- }
-
- // optional int64 records_completed = 3;
- private long recordsCompleted_ ;
- /**
- * <code>optional int64 records_completed = 3;</code>
- */
- public boolean hasRecordsCompleted() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- /**
- * <code>optional int64 records_completed = 3;</code>
- */
- public long getRecordsCompleted() {
- return recordsCompleted_;
- }
- /**
- * <code>optional int64 records_completed = 3;</code>
- */
- public Builder setRecordsCompleted(long value) {
- bitField0_ |= 0x00000004;
- recordsCompleted_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int64 records_completed = 3;</code>
- */
- public Builder clearRecordsCompleted() {
- bitField0_ = (bitField0_ & ~0x00000004);
- recordsCompleted_ = 0L;
- onChanged();
- return this;
- }
-
- // optional int32 estimated_completion_percentage = 4;
- private int estimatedCompletionPercentage_ ;
- /**
- * <code>optional int32 estimated_completion_percentage = 4;</code>
- */
- public boolean hasEstimatedCompletionPercentage() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
- }
- /**
- * <code>optional int32 estimated_completion_percentage = 4;</code>
- */
- public int getEstimatedCompletionPercentage() {
- return estimatedCompletionPercentage_;
- }
- /**
- * <code>optional int32 estimated_completion_percentage = 4;</code>
- */
- public Builder setEstimatedCompletionPercentage(int value) {
- bitField0_ |= 0x00000008;
- estimatedCompletionPercentage_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int32 estimated_completion_percentage = 4;</code>
- */
- public Builder clearEstimatedCompletionPercentage() {
- bitField0_ = (bitField0_ & ~0x00000008);
- estimatedCompletionPercentage_ = 0;
- onChanged();
- return this;
- }
-
- // optional .exec.bit.FragmentStatus.FragmentState state = 5;
- private org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState state_ = org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState.SENDING;
- /**
- * <code>optional .exec.bit.FragmentStatus.FragmentState state = 5;</code>
- */
- public boolean hasState() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- /**
- * <code>optional .exec.bit.FragmentStatus.FragmentState state = 5;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState getState() {
- return state_;
- }
- /**
- * <code>optional .exec.bit.FragmentStatus.FragmentState state = 5;</code>
- */
- public Builder setState(org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000010;
- state_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentStatus.FragmentState state = 5;</code>
- */
- public Builder clearState() {
- bitField0_ = (bitField0_ & ~0x00000010);
- state_ = org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState.SENDING;
- onChanged();
- return this;
- }
-
- // optional int64 data_processed = 6;
- private long dataProcessed_ ;
- /**
- * <code>optional int64 data_processed = 6;</code>
- */
- public boolean hasDataProcessed() {
- return ((bitField0_ & 0x00000020) == 0x00000020);
- }
- /**
- * <code>optional int64 data_processed = 6;</code>
- */
- public long getDataProcessed() {
- return dataProcessed_;
- }
- /**
- * <code>optional int64 data_processed = 6;</code>
- */
- public Builder setDataProcessed(long value) {
- bitField0_ |= 0x00000020;
- dataProcessed_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int64 data_processed = 6;</code>
- */
- public Builder clearDataProcessed() {
- bitField0_ = (bitField0_ & ~0x00000020);
- dataProcessed_ = 0L;
- onChanged();
- return this;
- }
-
- // optional .exec.bit.FragmentHandle handle = 7;
- private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder> handleBuilder_;
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public boolean hasHandle() {
- return ((bitField0_ & 0x00000040) == 0x00000040);
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
- if (handleBuilder_ == null) {
- return handle_;
- } else {
- return handleBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public Builder setHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
- if (handleBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- handle_ = value;
- onChanged();
- } else {
- handleBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000040;
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public Builder setHandle(
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder builderForValue) {
- if (handleBuilder_ == null) {
- handle_ = builderForValue.build();
- onChanged();
- } else {
- handleBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000040;
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public Builder mergeHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
- if (handleBuilder_ == null) {
- if (((bitField0_ & 0x00000040) == 0x00000040) &&
- handle_ != org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance()) {
- handle_ =
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle.newBuilder(handle_).mergeFrom(value).buildPartial();
- } else {
- handle_ = value;
- }
- onChanged();
- } else {
- handleBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000040;
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public Builder clearHandle() {
- if (handleBuilder_ == null) {
- handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- onChanged();
- } else {
- handleBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000040);
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder getHandleBuilder() {
- bitField0_ |= 0x00000040;
- onChanged();
- return getHandleFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
- if (handleBuilder_ != null) {
- return handleBuilder_.getMessageOrBuilder();
- } else {
- return handle_;
- }
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>
- getHandleFieldBuilder() {
- if (handleBuilder_ == null) {
- handleBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>(
- handle_,
- getParentForChildren(),
- isClean());
- handle_ = null;
- }
- return handleBuilder_;
- }
-
- // optional .exec.shared.DrillPBError error = 8;
- private org.apache.drill.exec.proto.UserBitShared.DrillPBError error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> errorBuilder_;
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public boolean hasError() {
- return ((bitField0_ & 0x00000080) == 0x00000080);
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError() {
- if (errorBuilder_ == null) {
- return error_;
- } else {
- return errorBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public Builder setError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
- if (errorBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- error_ = value;
- onChanged();
- } else {
- errorBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000080;
- return this;
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public Builder setError(
- org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
- if (errorBuilder_ == null) {
- error_ = builderForValue.build();
- onChanged();
- } else {
- errorBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000080;
- return this;
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public Builder mergeError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
- if (errorBuilder_ == null) {
- if (((bitField0_ & 0x00000080) == 0x00000080) &&
- error_ != org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance()) {
- error_ =
- org.apache.drill.exec.proto.UserBitShared.DrillPBError.newBuilder(error_).mergeFrom(value).buildPartial();
- } else {
- error_ = value;
- }
- onChanged();
- } else {
- errorBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000080;
- return this;
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public Builder clearError() {
- if (errorBuilder_ == null) {
- error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
- onChanged();
- } else {
- errorBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000080);
- return this;
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder getErrorBuilder() {
- bitField0_ |= 0x00000080;
- onChanged();
- return getErrorFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder() {
- if (errorBuilder_ != null) {
- return errorBuilder_.getMessageOrBuilder();
- } else {
- return error_;
- }
- }
- /**
- * <code>optional .exec.shared.DrillPBError error = 8;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder>
- getErrorFieldBuilder() {
- if (errorBuilder_ == null) {
- errorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder>(
- error_,
- getParentForChildren(),
- isClean());
- error_ = null;
- }
- return errorBuilder_;
- }
-
- // optional int64 running_time = 9;
- private long runningTime_ ;
- /**
- * <code>optional int64 running_time = 9;</code>
- */
- public boolean hasRunningTime() {
- return ((bitField0_ & 0x00000100) == 0x00000100);
- }
- /**
- * <code>optional int64 running_time = 9;</code>
- */
- public long getRunningTime() {
- return runningTime_;
- }
- /**
- * <code>optional int64 running_time = 9;</code>
- */
- public Builder setRunningTime(long value) {
- bitField0_ |= 0x00000100;
- runningTime_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int64 running_time = 9;</code>
- */
- public Builder clearRunningTime() {
- bitField0_ = (bitField0_ & ~0x00000100);
- runningTime_ = 0L;
- onChanged();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:exec.bit.FragmentStatus)
- }
-
- static {
- defaultInstance = new FragmentStatus(true);
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:exec.bit.FragmentStatus)
- }
-
- public interface FragmentRecordBatchOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
-
- // optional .exec.bit.FragmentHandle handle = 1;
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- boolean hasHandle();
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle();
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder();
-
- // optional int32 sending_major_fragment_id = 2;
- /**
- * <code>optional int32 sending_major_fragment_id = 2;</code>
- */
- boolean hasSendingMajorFragmentId();
- /**
- * <code>optional int32 sending_major_fragment_id = 2;</code>
- */
- int getSendingMajorFragmentId();
-
- // optional int32 sending_minor_fragment_id = 3;
- /**
- * <code>optional int32 sending_minor_fragment_id = 3;</code>
- */
- boolean hasSendingMinorFragmentId();
- /**
- * <code>optional int32 sending_minor_fragment_id = 3;</code>
- */
- int getSendingMinorFragmentId();
-
- // optional .exec.shared.RecordBatchDef def = 4;
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- boolean hasDef();
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef();
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder();
-
- // optional bool isLastBatch = 5;
- /**
- * <code>optional bool isLastBatch = 5;</code>
- */
- boolean hasIsLastBatch();
- /**
- * <code>optional bool isLastBatch = 5;</code>
- */
- boolean getIsLastBatch();
- }
- /**
- * Protobuf type {@code exec.bit.FragmentRecordBatch}
- */
- public static final class FragmentRecordBatch extends
- com.google.protobuf.GeneratedMessage
- implements FragmentRecordBatchOrBuilder {
- // Use FragmentRecordBatch.newBuilder() to construct.
- private FragmentRecordBatch(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
- super(builder);
- this.unknownFields = builder.getUnknownFields();
- }
- private FragmentRecordBatch(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
- private static final FragmentRecordBatch defaultInstance;
- public static FragmentRecordBatch getDefaultInstance() {
- return defaultInstance;
- }
-
- public FragmentRecordBatch getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- private final com.google.protobuf.UnknownFieldSet unknownFields;
- @java.lang.Override
- public final com.google.protobuf.UnknownFieldSet
- getUnknownFields() {
- return this.unknownFields;
- }
- private FragmentRecordBatch(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- initFields();
- int mutable_bitField0_ = 0;
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder();
- try {
- boolean done = false;
- while (!done) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- done = true;
- break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- done = true;
- }
- break;
- }
- case 10: {
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder subBuilder = null;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- subBuilder = handle_.toBuilder();
- }
- handle_ = input.readMessage(org.apache.drill.exec.proto.ExecProtos.FragmentHandle.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(handle_);
- handle_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000001;
- break;
- }
- case 16: {
- bitField0_ |= 0x00000002;
- sendingMajorFragmentId_ = input.readInt32();
- break;
- }
- case 24: {
- bitField0_ |= 0x00000004;
- sendingMinorFragmentId_ = input.readInt32();
- break;
- }
- case 34: {
- org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder subBuilder = null;
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- subBuilder = def_.toBuilder();
- }
- def_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(def_);
- def_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000008;
- break;
- }
- case 40: {
- bitField0_ |= 0x00000010;
- isLastBatch_ = input.readBool();
- break;
- }
- }
- }
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- throw e.setUnfinishedMessage(this);
- } catch (java.io.IOException e) {
- throw new com.google.protobuf.InvalidProtocolBufferException(
- e.getMessage()).setUnfinishedMessage(this);
- } finally {
- this.unknownFields = unknownFields.build();
- makeExtensionsImmutable();
- }
- }
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentRecordBatch_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentRecordBatch_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch.class, org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch.Builder.class);
- }
-
- public static com.google.protobuf.Parser<FragmentRecordBatch> PARSER =
- new com.google.protobuf.AbstractParser<FragmentRecordBatch>() {
- public FragmentRecordBatch parsePartialFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return new FragmentRecordBatch(input, extensionRegistry);
- }
- };
-
- @java.lang.Override
- public com.google.protobuf.Parser<FragmentRecordBatch> getParserForType() {
- return PARSER;
- }
-
- private int bitField0_;
- // optional .exec.bit.FragmentHandle handle = 1;
- public static final int HANDLE_FIELD_NUMBER = 1;
- private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_;
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public boolean hasHandle() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
- return handle_;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
- return handle_;
- }
-
- // optional int32 sending_major_fragment_id = 2;
- public static final int SENDING_MAJOR_FRAGMENT_ID_FIELD_NUMBER = 2;
- private int sendingMajorFragmentId_;
- /**
- * <code>optional int32 sending_major_fragment_id = 2;</code>
- */
- public boolean hasSendingMajorFragmentId() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional int32 sending_major_fragment_id = 2;</code>
- */
- public int getSendingMajorFragmentId() {
- return sendingMajorFragmentId_;
- }
-
- // optional int32 sending_minor_fragment_id = 3;
- public static final int SENDING_MINOR_FRAGMENT_ID_FIELD_NUMBER = 3;
- private int sendingMinorFragmentId_;
- /**
- * <code>optional int32 sending_minor_fragment_id = 3;</code>
- */
- public boolean hasSendingMinorFragmentId() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- /**
- * <code>optional int32 sending_minor_fragment_id = 3;</code>
- */
- public int getSendingMinorFragmentId() {
- return sendingMinorFragmentId_;
- }
-
- // optional .exec.shared.RecordBatchDef def = 4;
- public static final int DEF_FIELD_NUMBER = 4;
- private org.apache.drill.exec.proto.UserBitShared.RecordBatchDef def_;
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public boolean hasDef() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef() {
- return def_;
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder() {
- return def_;
- }
-
- // optional bool isLastBatch = 5;
- public static final int ISLASTBATCH_FIELD_NUMBER = 5;
- private boolean isLastBatch_;
- /**
- * <code>optional bool isLastBatch = 5;</code>
- */
- public boolean hasIsLastBatch() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- /**
- * <code>optional bool isLastBatch = 5;</code>
- */
- public boolean getIsLastBatch() {
- return isLastBatch_;
- }
-
- private void initFields() {
- handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- sendingMajorFragmentId_ = 0;
- sendingMinorFragmentId_ = 0;
- def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
- isLastBatch_ = false;
- }
- private byte memoizedIsInitialized = -1;
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) return isInitialized == 1;
-
- memoizedIsInitialized = 1;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeMessage(1, handle_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- output.writeInt32(2, sendingMajorFragmentId_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- output.writeInt32(3, sendingMinorFragmentId_);
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- output.writeMessage(4, def_);
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- output.writeBool(5, isLastBatch_);
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(1, handle_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(2, sendingMajorFragmentId_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(3, sendingMinorFragmentId_);
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(4, def_);
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(5, isLastBatch_);
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @java.lang.Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
- }
-
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- @java.lang.Override
- protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
- }
- /**
- * Protobuf type {@code exec.bit.FragmentRecordBatch}
- */
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatchOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentRecordBatch_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentRecordBatch_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch.class, org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch.Builder.class);
- }
-
- // Construct using org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- getHandleFieldBuilder();
- getDefFieldBuilder();
- }
- }
- private static Builder create() {
- return new Builder();
- }
-
- public Builder clear() {
- super.clear();
- if (handleBuilder_ == null) {
- handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- } else {
- handleBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000001);
- sendingMajorFragmentId_ = 0;
- bitField0_ = (bitField0_ & ~0x00000002);
- sendingMinorFragmentId_ = 0;
- bitField0_ = (bitField0_ & ~0x00000004);
- if (defBuilder_ == null) {
- def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
- } else {
- defBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000008);
- isLastBatch_ = false;
- bitField0_ = (bitField0_ & ~0x00000010);
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(buildPartial());
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_FragmentRecordBatch_descriptor;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch getDefaultInstanceForType() {
- return org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch.getDefaultInstance();
- }
-
- public org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch build() {
- org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch result = buildPartial();
- if (!result.isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return result;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch buildPartial() {
- org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch result = new org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch(this);
- int from_bitField0_ = bitField0_;
- int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
- to_bitField0_ |= 0x00000001;
- }
- if (handleBuilder_ == null) {
- result.handle_ = handle_;
- } else {
- result.handle_ = handleBuilder_.build();
- }
- if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
- to_bitField0_ |= 0x00000002;
- }
- result.sendingMajorFragmentId_ = sendingMajorFragmentId_;
- if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
- to_bitField0_ |= 0x00000004;
- }
- result.sendingMinorFragmentId_ = sendingMinorFragmentId_;
- if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
- to_bitField0_ |= 0x00000008;
- }
- if (defBuilder_ == null) {
- result.def_ = def_;
- } else {
- result.def_ = defBuilder_.build();
- }
- if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
- to_bitField0_ |= 0x00000010;
- }
- result.isLastBatch_ = isLastBatch_;
- result.bitField0_ = to_bitField0_;
- onBuilt();
- return result;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch) {
- return mergeFrom((org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch other) {
- if (other == org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch.getDefaultInstance()) return this;
- if (other.hasHandle()) {
- mergeHandle(other.getHandle());
- }
- if (other.hasSendingMajorFragmentId()) {
- setSendingMajorFragmentId(other.getSendingMajorFragmentId());
- }
- if (other.hasSendingMinorFragmentId()) {
- setSendingMinorFragmentId(other.getSendingMinorFragmentId());
- }
- if (other.hasDef()) {
- mergeDef(other.getDef());
- }
- if (other.hasIsLastBatch()) {
- setIsLastBatch(other.getIsLastBatch());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public final boolean isInitialized() {
- return true;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch parsedMessage = null;
- try {
- parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch) e.getUnfinishedMessage();
- throw e;
- } finally {
- if (parsedMessage != null) {
- mergeFrom(parsedMessage);
- }
- }
- return this;
- }
- private int bitField0_;
-
- // optional .exec.bit.FragmentHandle handle = 1;
- private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder> handleBuilder_;
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public boolean hasHandle() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
- if (handleBuilder_ == null) {
- return handle_;
- } else {
- return handleBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public Builder setHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
- if (handleBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- handle_ = value;
- onChanged();
- } else {
- handleBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000001;
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public Builder setHandle(
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder builderForValue) {
- if (handleBuilder_ == null) {
- handle_ = builderForValue.build();
- onChanged();
- } else {
- handleBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000001;
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public Builder mergeHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
- if (handleBuilder_ == null) {
- if (((bitField0_ & 0x00000001) == 0x00000001) &&
- handle_ != org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance()) {
- handle_ =
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle.newBuilder(handle_).mergeFrom(value).buildPartial();
- } else {
- handle_ = value;
- }
- onChanged();
- } else {
- handleBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000001;
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public Builder clearHandle() {
- if (handleBuilder_ == null) {
- handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- onChanged();
- } else {
- handleBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000001);
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder getHandleBuilder() {
- bitField0_ |= 0x00000001;
- onChanged();
- return getHandleFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
- if (handleBuilder_ != null) {
- return handleBuilder_.getMessageOrBuilder();
- } else {
- return handle_;
- }
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>
- getHandleFieldBuilder() {
- if (handleBuilder_ == null) {
- handleBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>(
- handle_,
- getParentForChildren(),
- isClean());
- handle_ = null;
- }
- return handleBuilder_;
- }
-
- // optional int32 sending_major_fragment_id = 2;
- private int sendingMajorFragmentId_ ;
- /**
- * <code>optional int32 sending_major_fragment_id = 2;</code>
- */
- public boolean hasSendingMajorFragmentId() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional int32 sending_major_fragment_id = 2;</code>
- */
- public int getSendingMajorFragmentId() {
- return sendingMajorFragmentId_;
- }
- /**
- * <code>optional int32 sending_major_fragment_id = 2;</code>
- */
- public Builder setSendingMajorFragmentId(int value) {
- bitField0_ |= 0x00000002;
- sendingMajorFragmentId_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int32 sending_major_fragment_id = 2;</code>
- */
- public Builder clearSendingMajorFragmentId() {
- bitField0_ = (bitField0_ & ~0x00000002);
- sendingMajorFragmentId_ = 0;
- onChanged();
- return this;
- }
-
- // optional int32 sending_minor_fragment_id = 3;
- private int sendingMinorFragmentId_ ;
- /**
- * <code>optional int32 sending_minor_fragment_id = 3;</code>
- */
- public boolean hasSendingMinorFragmentId() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- /**
- * <code>optional int32 sending_minor_fragment_id = 3;</code>
- */
- public int getSendingMinorFragmentId() {
- return sendingMinorFragmentId_;
- }
- /**
- * <code>optional int32 sending_minor_fragment_id = 3;</code>
- */
- public Builder setSendingMinorFragmentId(int value) {
- bitField0_ |= 0x00000004;
- sendingMinorFragmentId_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int32 sending_minor_fragment_id = 3;</code>
- */
- public Builder clearSendingMinorFragmentId() {
- bitField0_ = (bitField0_ & ~0x00000004);
- sendingMinorFragmentId_ = 0;
- onChanged();
- return this;
- }
-
- // optional .exec.shared.RecordBatchDef def = 4;
- private org.apache.drill.exec.proto.UserBitShared.RecordBatchDef def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.UserBitShared.RecordBatchDef, org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder, org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder> defBuilder_;
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public boolean hasDef() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef() {
- if (defBuilder_ == null) {
- return def_;
- } else {
- return defBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public Builder setDef(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef value) {
- if (defBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- def_ = value;
- onChanged();
- } else {
- defBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000008;
- return this;
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public Builder setDef(
- org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder builderForValue) {
- if (defBuilder_ == null) {
- def_ = builderForValue.build();
- onChanged();
- } else {
- defBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000008;
- return this;
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public Builder mergeDef(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef value) {
- if (defBuilder_ == null) {
- if (((bitField0_ & 0x00000008) == 0x00000008) &&
- def_ != org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance()) {
- def_ =
- org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.newBuilder(def_).mergeFrom(value).buildPartial();
- } else {
- def_ = value;
- }
- onChanged();
- } else {
- defBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000008;
- return this;
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public Builder clearDef() {
- if (defBuilder_ == null) {
- def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
- onChanged();
- } else {
- defBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000008);
- return this;
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder getDefBuilder() {
- bitField0_ |= 0x00000008;
- onChanged();
- return getDefFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- public org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder() {
- if (defBuilder_ != null) {
- return defBuilder_.getMessageOrBuilder();
- } else {
- return def_;
- }
- }
- /**
- * <code>optional .exec.shared.RecordBatchDef def = 4;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.UserBitShared.RecordBatchDef, org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder, org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder>
- getDefFieldBuilder() {
- if (defBuilder_ == null) {
- defBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.UserBitShared.RecordBatchDef, org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder, org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder>(
- def_,
- getParentForChildren(),
- isClean());
- def_ = null;
- }
- return defBuilder_;
- }
-
- // optional bool isLastBatch = 5;
- private boolean isLastBatch_ ;
- /**
- * <code>optional bool isLastBatch = 5;</code>
- */
- public boolean hasIsLastBatch() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- /**
- * <code>optional bool isLastBatch = 5;</code>
- */
- public boolean getIsLastBatch() {
- return isLastBatch_;
- }
- /**
- * <code>optional bool isLastBatch = 5;</code>
- */
- public Builder setIsLastBatch(boolean value) {
- bitField0_ |= 0x00000010;
- isLastBatch_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional bool isLastBatch = 5;</code>
- */
- public Builder clearIsLastBatch() {
- bitField0_ = (bitField0_ & ~0x00000010);
- isLastBatch_ = false;
- onChanged();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:exec.bit.FragmentRecordBatch)
- }
-
- static {
- defaultInstance = new FragmentRecordBatch(true);
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:exec.bit.FragmentRecordBatch)
- }
-
- public interface PlanFragmentOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
-
- // optional .exec.bit.FragmentHandle handle = 1;
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- boolean hasHandle();
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle();
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder();
-
- // optional float network_cost = 4;
- /**
- * <code>optional float network_cost = 4;</code>
- */
- boolean hasNetworkCost();
- /**
- * <code>optional float network_cost = 4;</code>
- */
- float getNetworkCost();
-
- // optional float cpu_cost = 5;
- /**
- * <code>optional float cpu_cost = 5;</code>
- */
- boolean hasCpuCost();
- /**
- * <code>optional float cpu_cost = 5;</code>
- */
- float getCpuCost();
-
- // optional float disk_cost = 6;
- /**
- * <code>optional float disk_cost = 6;</code>
- */
- boolean hasDiskCost();
- /**
- * <code>optional float disk_cost = 6;</code>
- */
- float getDiskCost();
-
- // optional float memory_cost = 7;
- /**
- * <code>optional float memory_cost = 7;</code>
- */
- boolean hasMemoryCost();
- /**
- * <code>optional float memory_cost = 7;</code>
- */
- float getMemoryCost();
-
- // optional string fragment_json = 8;
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- boolean hasFragmentJson();
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- java.lang.String getFragmentJson();
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- com.google.protobuf.ByteString
- getFragmentJsonBytes();
-
- // optional .exec.DrillbitEndpoint assignment = 10;
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- boolean hasAssignment();
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getAssignment();
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getAssignmentOrBuilder();
-
- // optional bool leaf_fragment = 9;
- /**
- * <code>optional bool leaf_fragment = 9;</code>
- */
- boolean hasLeafFragment();
- /**
- * <code>optional bool leaf_fragment = 9;</code>
- */
- boolean getLeafFragment();
-
- // optional .exec.DrillbitEndpoint foreman = 11;
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- boolean hasForeman();
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getForeman();
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getForemanOrBuilder();
- }
- /**
- * Protobuf type {@code exec.bit.PlanFragment}
- */
- public static final class PlanFragment extends
- com.google.protobuf.GeneratedMessage
- implements PlanFragmentOrBuilder {
- // Use PlanFragment.newBuilder() to construct.
- private PlanFragment(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
- super(builder);
- this.unknownFields = builder.getUnknownFields();
- }
- private PlanFragment(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
- private static final PlanFragment defaultInstance;
- public static PlanFragment getDefaultInstance() {
- return defaultInstance;
- }
-
- public PlanFragment getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- private final com.google.protobuf.UnknownFieldSet unknownFields;
- @java.lang.Override
- public final com.google.protobuf.UnknownFieldSet
- getUnknownFields() {
- return this.unknownFields;
- }
- private PlanFragment(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- initFields();
- int mutable_bitField0_ = 0;
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder();
- try {
- boolean done = false;
- while (!done) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- done = true;
- break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- done = true;
- }
- break;
- }
- case 10: {
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder subBuilder = null;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- subBuilder = handle_.toBuilder();
- }
- handle_ = input.readMessage(org.apache.drill.exec.proto.ExecProtos.FragmentHandle.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(handle_);
- handle_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000001;
- break;
- }
- case 37: {
- bitField0_ |= 0x00000002;
- networkCost_ = input.readFloat();
- break;
- }
- case 45: {
- bitField0_ |= 0x00000004;
- cpuCost_ = input.readFloat();
- break;
- }
- case 53: {
- bitField0_ |= 0x00000008;
- diskCost_ = input.readFloat();
- break;
- }
- case 61: {
- bitField0_ |= 0x00000010;
- memoryCost_ = input.readFloat();
- break;
- }
- case 66: {
- bitField0_ |= 0x00000020;
- fragmentJson_ = input.readBytes();
- break;
- }
- case 72: {
- bitField0_ |= 0x00000080;
- leafFragment_ = input.readBool();
- break;
- }
- case 82: {
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- subBuilder = assignment_.toBuilder();
- }
- assignment_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(assignment_);
- assignment_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000040;
- break;
- }
- case 90: {
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
- if (((bitField0_ & 0x00000100) == 0x00000100)) {
- subBuilder = foreman_.toBuilder();
- }
- foreman_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(foreman_);
- foreman_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000100;
- break;
- }
- }
- }
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- throw e.setUnfinishedMessage(this);
- } catch (java.io.IOException e) {
- throw new com.google.protobuf.InvalidProtocolBufferException(
- e.getMessage()).setUnfinishedMessage(this);
- } finally {
- this.unknownFields = unknownFields.build();
- makeExtensionsImmutable();
- }
- }
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_PlanFragment_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_PlanFragment_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.PlanFragment.class, org.apache.drill.exec.proto.ExecProtos.PlanFragment.Builder.class);
- }
-
- public static com.google.protobuf.Parser<PlanFragment> PARSER =
- new com.google.protobuf.AbstractParser<PlanFragment>() {
- public PlanFragment parsePartialFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return new PlanFragment(input, extensionRegistry);
- }
- };
-
- @java.lang.Override
- public com.google.protobuf.Parser<PlanFragment> getParserForType() {
- return PARSER;
- }
-
- private int bitField0_;
- // optional .exec.bit.FragmentHandle handle = 1;
- public static final int HANDLE_FIELD_NUMBER = 1;
- private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_;
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public boolean hasHandle() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
- return handle_;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
- return handle_;
- }
-
- // optional float network_cost = 4;
- public static final int NETWORK_COST_FIELD_NUMBER = 4;
- private float networkCost_;
- /**
- * <code>optional float network_cost = 4;</code>
- */
- public boolean hasNetworkCost() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional float network_cost = 4;</code>
- */
- public float getNetworkCost() {
- return networkCost_;
- }
-
- // optional float cpu_cost = 5;
- public static final int CPU_COST_FIELD_NUMBER = 5;
- private float cpuCost_;
- /**
- * <code>optional float cpu_cost = 5;</code>
- */
- public boolean hasCpuCost() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- /**
- * <code>optional float cpu_cost = 5;</code>
- */
- public float getCpuCost() {
- return cpuCost_;
- }
-
- // optional float disk_cost = 6;
- public static final int DISK_COST_FIELD_NUMBER = 6;
- private float diskCost_;
- /**
- * <code>optional float disk_cost = 6;</code>
- */
- public boolean hasDiskCost() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
- }
- /**
- * <code>optional float disk_cost = 6;</code>
- */
- public float getDiskCost() {
- return diskCost_;
- }
-
- // optional float memory_cost = 7;
- public static final int MEMORY_COST_FIELD_NUMBER = 7;
- private float memoryCost_;
- /**
- * <code>optional float memory_cost = 7;</code>
- */
- public boolean hasMemoryCost() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- /**
- * <code>optional float memory_cost = 7;</code>
- */
- public float getMemoryCost() {
- return memoryCost_;
- }
-
- // optional string fragment_json = 8;
- public static final int FRAGMENT_JSON_FIELD_NUMBER = 8;
- private java.lang.Object fragmentJson_;
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- public boolean hasFragmentJson() {
- return ((bitField0_ & 0x00000020) == 0x00000020);
- }
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- public java.lang.String getFragmentJson() {
- java.lang.Object ref = fragmentJson_;
- if (ref instanceof java.lang.String) {
- return (java.lang.String) ref;
- } else {
- com.google.protobuf.ByteString bs =
- (com.google.protobuf.ByteString) ref;
- java.lang.String s = bs.toStringUtf8();
- if (bs.isValidUtf8()) {
- fragmentJson_ = s;
- }
- return s;
- }
- }
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- public com.google.protobuf.ByteString
- getFragmentJsonBytes() {
- java.lang.Object ref = fragmentJson_;
- if (ref instanceof java.lang.String) {
- com.google.protobuf.ByteString b =
- com.google.protobuf.ByteString.copyFromUtf8(
- (java.lang.String) ref);
- fragmentJson_ = b;
- return b;
- } else {
- return (com.google.protobuf.ByteString) ref;
- }
- }
-
- // optional .exec.DrillbitEndpoint assignment = 10;
- public static final int ASSIGNMENT_FIELD_NUMBER = 10;
- private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint assignment_;
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public boolean hasAssignment() {
- return ((bitField0_ & 0x00000040) == 0x00000040);
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getAssignment() {
- return assignment_;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getAssignmentOrBuilder() {
- return assignment_;
- }
-
- // optional bool leaf_fragment = 9;
- public static final int LEAF_FRAGMENT_FIELD_NUMBER = 9;
- private boolean leafFragment_;
- /**
- * <code>optional bool leaf_fragment = 9;</code>
- */
- public boolean hasLeafFragment() {
- return ((bitField0_ & 0x00000080) == 0x00000080);
- }
- /**
- * <code>optional bool leaf_fragment = 9;</code>
- */
- public boolean getLeafFragment() {
- return leafFragment_;
- }
-
- // optional .exec.DrillbitEndpoint foreman = 11;
- public static final int FOREMAN_FIELD_NUMBER = 11;
- private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint foreman_;
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public boolean hasForeman() {
- return ((bitField0_ & 0x00000100) == 0x00000100);
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getForeman() {
- return foreman_;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getForemanOrBuilder() {
- return foreman_;
- }
-
- private void initFields() {
- handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- networkCost_ = 0F;
- cpuCost_ = 0F;
- diskCost_ = 0F;
- memoryCost_ = 0F;
- fragmentJson_ = "";
- assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- leafFragment_ = false;
- foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- }
- private byte memoizedIsInitialized = -1;
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) return isInitialized == 1;
-
- memoizedIsInitialized = 1;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeMessage(1, handle_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- output.writeFloat(4, networkCost_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- output.writeFloat(5, cpuCost_);
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- output.writeFloat(6, diskCost_);
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- output.writeFloat(7, memoryCost_);
- }
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
- output.writeBytes(8, getFragmentJsonBytes());
- }
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
- output.writeBool(9, leafFragment_);
- }
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- output.writeMessage(10, assignment_);
- }
- if (((bitField0_ & 0x00000100) == 0x00000100)) {
- output.writeMessage(11, foreman_);
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(1, handle_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- size += com.google.protobuf.CodedOutputStream
- .computeFloatSize(4, networkCost_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- size += com.google.protobuf.CodedOutputStream
- .computeFloatSize(5, cpuCost_);
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- size += com.google.protobuf.CodedOutputStream
- .computeFloatSize(6, diskCost_);
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- size += com.google.protobuf.CodedOutputStream
- .computeFloatSize(7, memoryCost_);
- }
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(8, getFragmentJsonBytes());
- }
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(9, leafFragment_);
- }
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(10, assignment_);
- }
- if (((bitField0_ & 0x00000100) == 0x00000100)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(11, foreman_);
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @java.lang.Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
- }
-
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.PlanFragment parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.drill.exec.proto.ExecProtos.PlanFragment prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- @java.lang.Override
- protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
- }
- /**
- * Protobuf type {@code exec.bit.PlanFragment}
- */
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.drill.exec.proto.ExecProtos.PlanFragmentOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_PlanFragment_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_PlanFragment_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.PlanFragment.class, org.apache.drill.exec.proto.ExecProtos.PlanFragment.Builder.class);
- }
-
- // Construct using org.apache.drill.exec.proto.ExecProtos.PlanFragment.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- getHandleFieldBuilder();
- getAssignmentFieldBuilder();
- getForemanFieldBuilder();
- }
- }
- private static Builder create() {
- return new Builder();
- }
-
- public Builder clear() {
- super.clear();
- if (handleBuilder_ == null) {
- handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- } else {
- handleBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000001);
- networkCost_ = 0F;
- bitField0_ = (bitField0_ & ~0x00000002);
- cpuCost_ = 0F;
- bitField0_ = (bitField0_ & ~0x00000004);
- diskCost_ = 0F;
- bitField0_ = (bitField0_ & ~0x00000008);
- memoryCost_ = 0F;
- bitField0_ = (bitField0_ & ~0x00000010);
- fragmentJson_ = "";
- bitField0_ = (bitField0_ & ~0x00000020);
- if (assignmentBuilder_ == null) {
- assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- } else {
- assignmentBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000040);
- leafFragment_ = false;
- bitField0_ = (bitField0_ & ~0x00000080);
- if (foremanBuilder_ == null) {
- foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- } else {
- foremanBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000100);
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(buildPartial());
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_PlanFragment_descriptor;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.PlanFragment getDefaultInstanceForType() {
- return org.apache.drill.exec.proto.ExecProtos.PlanFragment.getDefaultInstance();
- }
-
- public org.apache.drill.exec.proto.ExecProtos.PlanFragment build() {
- org.apache.drill.exec.proto.ExecProtos.PlanFragment result = buildPartial();
- if (!result.isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return result;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.PlanFragment buildPartial() {
- org.apache.drill.exec.proto.ExecProtos.PlanFragment result = new org.apache.drill.exec.proto.ExecProtos.PlanFragment(this);
- int from_bitField0_ = bitField0_;
- int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
- to_bitField0_ |= 0x00000001;
- }
- if (handleBuilder_ == null) {
- result.handle_ = handle_;
- } else {
- result.handle_ = handleBuilder_.build();
- }
- if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
- to_bitField0_ |= 0x00000002;
- }
- result.networkCost_ = networkCost_;
- if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
- to_bitField0_ |= 0x00000004;
- }
- result.cpuCost_ = cpuCost_;
- if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
- to_bitField0_ |= 0x00000008;
- }
- result.diskCost_ = diskCost_;
- if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
- to_bitField0_ |= 0x00000010;
- }
- result.memoryCost_ = memoryCost_;
- if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
- to_bitField0_ |= 0x00000020;
- }
- result.fragmentJson_ = fragmentJson_;
- if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
- to_bitField0_ |= 0x00000040;
- }
- if (assignmentBuilder_ == null) {
- result.assignment_ = assignment_;
- } else {
- result.assignment_ = assignmentBuilder_.build();
- }
- if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
- to_bitField0_ |= 0x00000080;
- }
- result.leafFragment_ = leafFragment_;
- if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
- to_bitField0_ |= 0x00000100;
- }
- if (foremanBuilder_ == null) {
- result.foreman_ = foreman_;
- } else {
- result.foreman_ = foremanBuilder_.build();
- }
- result.bitField0_ = to_bitField0_;
- onBuilt();
- return result;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.drill.exec.proto.ExecProtos.PlanFragment) {
- return mergeFrom((org.apache.drill.exec.proto.ExecProtos.PlanFragment)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(org.apache.drill.exec.proto.ExecProtos.PlanFragment other) {
- if (other == org.apache.drill.exec.proto.ExecProtos.PlanFragment.getDefaultInstance()) return this;
- if (other.hasHandle()) {
- mergeHandle(other.getHandle());
- }
- if (other.hasNetworkCost()) {
- setNetworkCost(other.getNetworkCost());
- }
- if (other.hasCpuCost()) {
- setCpuCost(other.getCpuCost());
- }
- if (other.hasDiskCost()) {
- setDiskCost(other.getDiskCost());
- }
- if (other.hasMemoryCost()) {
- setMemoryCost(other.getMemoryCost());
- }
- if (other.hasFragmentJson()) {
- bitField0_ |= 0x00000020;
- fragmentJson_ = other.fragmentJson_;
- onChanged();
- }
- if (other.hasAssignment()) {
- mergeAssignment(other.getAssignment());
- }
- if (other.hasLeafFragment()) {
- setLeafFragment(other.getLeafFragment());
- }
- if (other.hasForeman()) {
- mergeForeman(other.getForeman());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public final boolean isInitialized() {
- return true;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- org.apache.drill.exec.proto.ExecProtos.PlanFragment parsedMessage = null;
- try {
- parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.drill.exec.proto.ExecProtos.PlanFragment) e.getUnfinishedMessage();
- throw e;
- } finally {
- if (parsedMessage != null) {
- mergeFrom(parsedMessage);
- }
- }
- return this;
- }
- private int bitField0_;
-
- // optional .exec.bit.FragmentHandle handle = 1;
- private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder> handleBuilder_;
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public boolean hasHandle() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
- if (handleBuilder_ == null) {
- return handle_;
- } else {
- return handleBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public Builder setHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
- if (handleBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- handle_ = value;
- onChanged();
- } else {
- handleBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000001;
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public Builder setHandle(
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder builderForValue) {
- if (handleBuilder_ == null) {
- handle_ = builderForValue.build();
- onChanged();
- } else {
- handleBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000001;
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public Builder mergeHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
- if (handleBuilder_ == null) {
- if (((bitField0_ & 0x00000001) == 0x00000001) &&
- handle_ != org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance()) {
- handle_ =
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle.newBuilder(handle_).mergeFrom(value).buildPartial();
- } else {
- handle_ = value;
- }
- onChanged();
- } else {
- handleBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000001;
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public Builder clearHandle() {
- if (handleBuilder_ == null) {
- handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
- onChanged();
- } else {
- handleBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000001);
- return this;
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder getHandleBuilder() {
- bitField0_ |= 0x00000001;
- onChanged();
- return getHandleFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
- if (handleBuilder_ != null) {
- return handleBuilder_.getMessageOrBuilder();
- } else {
- return handle_;
- }
- }
- /**
- * <code>optional .exec.bit.FragmentHandle handle = 1;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>
- getHandleFieldBuilder() {
- if (handleBuilder_ == null) {
- handleBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder>(
- handle_,
- getParentForChildren(),
- isClean());
- handle_ = null;
- }
- return handleBuilder_;
- }
-
- // optional float network_cost = 4;
- private float networkCost_ ;
- /**
- * <code>optional float network_cost = 4;</code>
- */
- public boolean hasNetworkCost() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional float network_cost = 4;</code>
- */
- public float getNetworkCost() {
- return networkCost_;
- }
- /**
- * <code>optional float network_cost = 4;</code>
- */
- public Builder setNetworkCost(float value) {
- bitField0_ |= 0x00000002;
- networkCost_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional float network_cost = 4;</code>
- */
- public Builder clearNetworkCost() {
- bitField0_ = (bitField0_ & ~0x00000002);
- networkCost_ = 0F;
- onChanged();
- return this;
- }
-
- // optional float cpu_cost = 5;
- private float cpuCost_ ;
- /**
- * <code>optional float cpu_cost = 5;</code>
- */
- public boolean hasCpuCost() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- /**
- * <code>optional float cpu_cost = 5;</code>
- */
- public float getCpuCost() {
- return cpuCost_;
- }
- /**
- * <code>optional float cpu_cost = 5;</code>
- */
- public Builder setCpuCost(float value) {
- bitField0_ |= 0x00000004;
- cpuCost_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional float cpu_cost = 5;</code>
- */
- public Builder clearCpuCost() {
- bitField0_ = (bitField0_ & ~0x00000004);
- cpuCost_ = 0F;
- onChanged();
- return this;
- }
-
- // optional float disk_cost = 6;
- private float diskCost_ ;
- /**
- * <code>optional float disk_cost = 6;</code>
- */
- public boolean hasDiskCost() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
- }
- /**
- * <code>optional float disk_cost = 6;</code>
- */
- public float getDiskCost() {
- return diskCost_;
- }
- /**
- * <code>optional float disk_cost = 6;</code>
- */
- public Builder setDiskCost(float value) {
- bitField0_ |= 0x00000008;
- diskCost_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional float disk_cost = 6;</code>
- */
- public Builder clearDiskCost() {
- bitField0_ = (bitField0_ & ~0x00000008);
- diskCost_ = 0F;
- onChanged();
- return this;
- }
-
- // optional float memory_cost = 7;
- private float memoryCost_ ;
- /**
- * <code>optional float memory_cost = 7;</code>
- */
- public boolean hasMemoryCost() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- /**
- * <code>optional float memory_cost = 7;</code>
- */
- public float getMemoryCost() {
- return memoryCost_;
- }
- /**
- * <code>optional float memory_cost = 7;</code>
- */
- public Builder setMemoryCost(float value) {
- bitField0_ |= 0x00000010;
- memoryCost_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional float memory_cost = 7;</code>
- */
- public Builder clearMemoryCost() {
- bitField0_ = (bitField0_ & ~0x00000010);
- memoryCost_ = 0F;
- onChanged();
- return this;
- }
-
- // optional string fragment_json = 8;
- private java.lang.Object fragmentJson_ = "";
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- public boolean hasFragmentJson() {
- return ((bitField0_ & 0x00000020) == 0x00000020);
- }
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- public java.lang.String getFragmentJson() {
- java.lang.Object ref = fragmentJson_;
- if (!(ref instanceof java.lang.String)) {
- java.lang.String s = ((com.google.protobuf.ByteString) ref)
- .toStringUtf8();
- fragmentJson_ = s;
- return s;
- } else {
- return (java.lang.String) ref;
- }
- }
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- public com.google.protobuf.ByteString
- getFragmentJsonBytes() {
- java.lang.Object ref = fragmentJson_;
- if (ref instanceof String) {
- com.google.protobuf.ByteString b =
- com.google.protobuf.ByteString.copyFromUtf8(
- (java.lang.String) ref);
- fragmentJson_ = b;
- return b;
- } else {
- return (com.google.protobuf.ByteString) ref;
- }
- }
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- public Builder setFragmentJson(
- java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000020;
- fragmentJson_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- public Builder clearFragmentJson() {
- bitField0_ = (bitField0_ & ~0x00000020);
- fragmentJson_ = getDefaultInstance().getFragmentJson();
- onChanged();
- return this;
- }
- /**
- * <code>optional string fragment_json = 8;</code>
- */
- public Builder setFragmentJsonBytes(
- com.google.protobuf.ByteString value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000020;
- fragmentJson_ = value;
- onChanged();
- return this;
- }
-
- // optional .exec.DrillbitEndpoint assignment = 10;
- private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> assignmentBuilder_;
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public boolean hasAssignment() {
- return ((bitField0_ & 0x00000040) == 0x00000040);
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getAssignment() {
- if (assignmentBuilder_ == null) {
- return assignment_;
- } else {
- return assignmentBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public Builder setAssignment(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
- if (assignmentBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- assignment_ = value;
- onChanged();
- } else {
- assignmentBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000040;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public Builder setAssignment(
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder builderForValue) {
- if (assignmentBuilder_ == null) {
- assignment_ = builderForValue.build();
- onChanged();
- } else {
- assignmentBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000040;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public Builder mergeAssignment(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
- if (assignmentBuilder_ == null) {
- if (((bitField0_ & 0x00000040) == 0x00000040) &&
- assignment_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
- assignment_ =
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(assignment_).mergeFrom(value).buildPartial();
- } else {
- assignment_ = value;
- }
- onChanged();
- } else {
- assignmentBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000040;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public Builder clearAssignment() {
- if (assignmentBuilder_ == null) {
- assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- onChanged();
- } else {
- assignmentBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000040);
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getAssignmentBuilder() {
- bitField0_ |= 0x00000040;
- onChanged();
- return getAssignmentFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getAssignmentOrBuilder() {
- if (assignmentBuilder_ != null) {
- return assignmentBuilder_.getMessageOrBuilder();
- } else {
- return assignment_;
- }
- }
- /**
- * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>
- getAssignmentFieldBuilder() {
- if (assignmentBuilder_ == null) {
- assignmentBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>(
- assignment_,
- getParentForChildren(),
- isClean());
- assignment_ = null;
- }
- return assignmentBuilder_;
- }
-
- // optional bool leaf_fragment = 9;
- private boolean leafFragment_ ;
- /**
- * <code>optional bool leaf_fragment = 9;</code>
- */
- public boolean hasLeafFragment() {
- return ((bitField0_ & 0x00000080) == 0x00000080);
- }
- /**
- * <code>optional bool leaf_fragment = 9;</code>
- */
- public boolean getLeafFragment() {
- return leafFragment_;
- }
- /**
- * <code>optional bool leaf_fragment = 9;</code>
- */
- public Builder setLeafFragment(boolean value) {
- bitField0_ |= 0x00000080;
- leafFragment_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional bool leaf_fragment = 9;</code>
- */
- public Builder clearLeafFragment() {
- bitField0_ = (bitField0_ & ~0x00000080);
- leafFragment_ = false;
- onChanged();
- return this;
- }
-
- // optional .exec.DrillbitEndpoint foreman = 11;
- private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> foremanBuilder_;
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public boolean hasForeman() {
- return ((bitField0_ & 0x00000100) == 0x00000100);
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getForeman() {
- if (foremanBuilder_ == null) {
- return foreman_;
- } else {
- return foremanBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public Builder setForeman(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
- if (foremanBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- foreman_ = value;
- onChanged();
- } else {
- foremanBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000100;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public Builder setForeman(
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder builderForValue) {
- if (foremanBuilder_ == null) {
- foreman_ = builderForValue.build();
- onChanged();
- } else {
- foremanBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000100;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public Builder mergeForeman(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
- if (foremanBuilder_ == null) {
- if (((bitField0_ & 0x00000100) == 0x00000100) &&
- foreman_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
- foreman_ =
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(foreman_).mergeFrom(value).buildPartial();
- } else {
- foreman_ = value;
- }
- onChanged();
- } else {
- foremanBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000100;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public Builder clearForeman() {
- if (foremanBuilder_ == null) {
- foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- onChanged();
- } else {
- foremanBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000100);
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getForemanBuilder() {
- bitField0_ |= 0x00000100;
- onChanged();
- return getForemanFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getForemanOrBuilder() {
- if (foremanBuilder_ != null) {
- return foremanBuilder_.getMessageOrBuilder();
- } else {
- return foreman_;
- }
- }
- /**
- * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>
- getForemanFieldBuilder() {
- if (foremanBuilder_ == null) {
- foremanBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>(
- foreman_,
- getParentForChildren(),
- isClean());
- foreman_ = null;
- }
- return foremanBuilder_;
- }
-
- // @@protoc_insertion_point(builder_scope:exec.bit.PlanFragment)
- }
-
- static {
- defaultInstance = new PlanFragment(true);
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:exec.bit.PlanFragment)
- }
-
public interface FragmentHandleOrBuilder
extends com.google.protobuf.MessageOrBuilder {
@@ -6267,715 +694,11 @@ public final class ExecProtos {
// @@protoc_insertion_point(class_scope:exec.bit.FragmentHandle)
}
- public interface WorkQueueStatusOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
-
- // optional .exec.DrillbitEndpoint endpoint = 1;
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- boolean hasEndpoint();
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint();
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder();
-
- // optional int32 queue_length = 2;
- /**
- * <code>optional int32 queue_length = 2;</code>
- */
- boolean hasQueueLength();
- /**
- * <code>optional int32 queue_length = 2;</code>
- */
- int getQueueLength();
-
- // optional int64 report_time = 3;
- /**
- * <code>optional int64 report_time = 3;</code>
- */
- boolean hasReportTime();
- /**
- * <code>optional int64 report_time = 3;</code>
- */
- long getReportTime();
- }
- /**
- * Protobuf type {@code exec.bit.WorkQueueStatus}
- */
- public static final class WorkQueueStatus extends
- com.google.protobuf.GeneratedMessage
- implements WorkQueueStatusOrBuilder {
- // Use WorkQueueStatus.newBuilder() to construct.
- private WorkQueueStatus(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
- super(builder);
- this.unknownFields = builder.getUnknownFields();
- }
- private WorkQueueStatus(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
- private static final WorkQueueStatus defaultInstance;
- public static WorkQueueStatus getDefaultInstance() {
- return defaultInstance;
- }
-
- public WorkQueueStatus getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- private final com.google.protobuf.UnknownFieldSet unknownFields;
- @java.lang.Override
- public final com.google.protobuf.UnknownFieldSet
- getUnknownFields() {
- return this.unknownFields;
- }
- private WorkQueueStatus(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- initFields();
- int mutable_bitField0_ = 0;
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder();
- try {
- boolean done = false;
- while (!done) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- done = true;
- break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- done = true;
- }
- break;
- }
- case 10: {
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- subBuilder = endpoint_.toBuilder();
- }
- endpoint_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(endpoint_);
- endpoint_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000001;
- break;
- }
- case 16: {
- bitField0_ |= 0x00000002;
- queueLength_ = input.readInt32();
- break;
- }
- case 24: {
- bitField0_ |= 0x00000004;
- reportTime_ = input.readInt64();
- break;
- }
- }
- }
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- throw e.setUnfinishedMessage(this);
- } catch (java.io.IOException e) {
- throw new com.google.protobuf.InvalidProtocolBufferException(
- e.getMessage()).setUnfinishedMessage(this);
- } finally {
- this.unknownFields = unknownFields.build();
- makeExtensionsImmutable();
- }
- }
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_WorkQueueStatus_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_WorkQueueStatus_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus.class, org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus.Builder.class);
- }
-
- public static com.google.protobuf.Parser<WorkQueueStatus> PARSER =
- new com.google.protobuf.AbstractParser<WorkQueueStatus>() {
- public WorkQueueStatus parsePartialFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return new WorkQueueStatus(input, extensionRegistry);
- }
- };
-
- @java.lang.Override
- public com.google.protobuf.Parser<WorkQueueStatus> getParserForType() {
- return PARSER;
- }
-
- private int bitField0_;
- // optional .exec.DrillbitEndpoint endpoint = 1;
- public static final int ENDPOINT_FIELD_NUMBER = 1;
- private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_;
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public boolean hasEndpoint() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
- return endpoint_;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
- return endpoint_;
- }
-
- // optional int32 queue_length = 2;
- public static final int QUEUE_LENGTH_FIELD_NUMBER = 2;
- private int queueLength_;
- /**
- * <code>optional int32 queue_length = 2;</code>
- */
- public boolean hasQueueLength() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional int32 queue_length = 2;</code>
- */
- public int getQueueLength() {
- return queueLength_;
- }
-
- // optional int64 report_time = 3;
- public static final int REPORT_TIME_FIELD_NUMBER = 3;
- private long reportTime_;
- /**
- * <code>optional int64 report_time = 3;</code>
- */
- public boolean hasReportTime() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- /**
- * <code>optional int64 report_time = 3;</code>
- */
- public long getReportTime() {
- return reportTime_;
- }
-
- private void initFields() {
- endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- queueLength_ = 0;
- reportTime_ = 0L;
- }
- private byte memoizedIsInitialized = -1;
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) return isInitialized == 1;
-
- memoizedIsInitialized = 1;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeMessage(1, endpoint_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- output.writeInt32(2, queueLength_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- output.writeInt64(3, reportTime_);
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(1, endpoint_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(2, queueLength_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt64Size(3, reportTime_);
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @java.lang.Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
- }
-
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- @java.lang.Override
- protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
- }
- /**
- * Protobuf type {@code exec.bit.WorkQueueStatus}
- */
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.drill.exec.proto.ExecProtos.WorkQueueStatusOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_WorkQueueStatus_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_WorkQueueStatus_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus.class, org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus.Builder.class);
- }
-
- // Construct using org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- getEndpointFieldBuilder();
- }
- }
- private static Builder create() {
- return new Builder();
- }
-
- public Builder clear() {
- super.clear();
- if (endpointBuilder_ == null) {
- endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- } else {
- endpointBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000001);
- queueLength_ = 0;
- bitField0_ = (bitField0_ & ~0x00000002);
- reportTime_ = 0L;
- bitField0_ = (bitField0_ & ~0x00000004);
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(buildPartial());
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return org.apache.drill.exec.proto.ExecProtos.internal_static_exec_bit_WorkQueueStatus_descriptor;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus getDefaultInstanceForType() {
- return org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus.getDefaultInstance();
- }
-
- public org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus build() {
- org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus result = buildPartial();
- if (!result.isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return result;
- }
-
- public org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus buildPartial() {
- org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus result = new org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus(this);
- int from_bitField0_ = bitField0_;
- int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
- to_bitField0_ |= 0x00000001;
- }
- if (endpointBuilder_ == null) {
- result.endpoint_ = endpoint_;
- } else {
- result.endpoint_ = endpointBuilder_.build();
- }
- if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
- to_bitField0_ |= 0x00000002;
- }
- result.queueLength_ = queueLength_;
- if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
- to_bitField0_ |= 0x00000004;
- }
- result.reportTime_ = reportTime_;
- result.bitField0_ = to_bitField0_;
- onBuilt();
- return result;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus) {
- return mergeFrom((org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus other) {
- if (other == org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus.getDefaultInstance()) return this;
- if (other.hasEndpoint()) {
- mergeEndpoint(other.getEndpoint());
- }
- if (other.hasQueueLength()) {
- setQueueLength(other.getQueueLength());
- }
- if (other.hasReportTime()) {
- setReportTime(other.getReportTime());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public final boolean isInitialized() {
- return true;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus parsedMessage = null;
- try {
- parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus) e.getUnfinishedMessage();
- throw e;
- } finally {
- if (parsedMessage != null) {
- mergeFrom(parsedMessage);
- }
- }
- return this;
- }
- private int bitField0_;
-
- // optional .exec.DrillbitEndpoint endpoint = 1;
- private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> endpointBuilder_;
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public boolean hasEndpoint() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
- if (endpointBuilder_ == null) {
- return endpoint_;
- } else {
- return endpointBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public Builder setEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
- if (endpointBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- endpoint_ = value;
- onChanged();
- } else {
- endpointBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000001;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public Builder setEndpoint(
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder builderForValue) {
- if (endpointBuilder_ == null) {
- endpoint_ = builderForValue.build();
- onChanged();
- } else {
- endpointBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000001;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public Builder mergeEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
- if (endpointBuilder_ == null) {
- if (((bitField0_ & 0x00000001) == 0x00000001) &&
- endpoint_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
- endpoint_ =
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(endpoint_).mergeFrom(value).buildPartial();
- } else {
- endpoint_ = value;
- }
- onChanged();
- } else {
- endpointBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000001;
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public Builder clearEndpoint() {
- if (endpointBuilder_ == null) {
- endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
- onChanged();
- } else {
- endpointBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000001);
- return this;
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getEndpointBuilder() {
- bitField0_ |= 0x00000001;
- onChanged();
- return getEndpointFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
- if (endpointBuilder_ != null) {
- return endpointBuilder_.getMessageOrBuilder();
- } else {
- return endpoint_;
- }
- }
- /**
- * <code>optional .exec.DrillbitEndpoint endpoint = 1;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>
- getEndpointFieldBuilder() {
- if (endpointBuilder_ == null) {
- endpointBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>(
- endpoint_,
- getParentForChildren(),
- isClean());
- endpoint_ = null;
- }
- return endpointBuilder_;
- }
-
- // optional int32 queue_length = 2;
- private int queueLength_ ;
- /**
- * <code>optional int32 queue_length = 2;</code>
- */
- public boolean hasQueueLength() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>optional int32 queue_length = 2;</code>
- */
- public int getQueueLength() {
- return queueLength_;
- }
- /**
- * <code>optional int32 queue_length = 2;</code>
- */
- public Builder setQueueLength(int value) {
- bitField0_ |= 0x00000002;
- queueLength_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int32 queue_length = 2;</code>
- */
- public Builder clearQueueLength() {
- bitField0_ = (bitField0_ & ~0x00000002);
- queueLength_ = 0;
- onChanged();
- return this;
- }
-
- // optional int64 report_time = 3;
- private long reportTime_ ;
- /**
- * <code>optional int64 report_time = 3;</code>
- */
- public boolean hasReportTime() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- /**
- * <code>optional int64 report_time = 3;</code>
- */
- public long getReportTime() {
- return reportTime_;
- }
- /**
- * <code>optional int64 report_time = 3;</code>
- */
- public Builder setReportTime(long value) {
- bitField0_ |= 0x00000004;
- reportTime_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional int64 report_time = 3;</code>
- */
- public Builder clearReportTime() {
- bitField0_ = (bitField0_ & ~0x00000004);
- reportTime_ = 0L;
- onChanged();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:exec.bit.WorkQueueStatus)
- }
-
- static {
- defaultInstance = new WorkQueueStatus(true);
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:exec.bit.WorkQueueStatus)
- }
-
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_exec_bit_BitHandshake_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_exec_bit_BitHandshake_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_exec_bit_BitBatchChunk_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_exec_bit_BitBatchChunk_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_exec_bit_BitStatus_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_exec_bit_BitStatus_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_exec_bit_FragmentStatus_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_exec_bit_FragmentStatus_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_exec_bit_FragmentRecordBatch_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_exec_bit_FragmentRecordBatch_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_exec_bit_PlanFragment_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_exec_bit_PlanFragment_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
internal_static_exec_bit_FragmentHandle_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_exec_bit_FragmentHandle_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_exec_bit_WorkQueueStatus_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_exec_bit_WorkQueueStatus_fieldAccessorTable;
public static com.google.protobuf.Descriptors.FileDescriptor
getDescriptor() {
@@ -6986,100 +709,23 @@ public final class ExecProtos {
static {
java.lang.String[] descriptorData = {
"\n\025ExecutionProtos.proto\022\010exec.bit\032\022Coord" +
- "ination.proto\032\023UserBitShared.proto\"M\n\014Bi" +
- "tHandshake\022\023\n\013rpc_version\030\001 \001(\005\022(\n\010endpo" +
- "int\030\002 \001(\0132\026.exec.DrillbitEndpoint\"\017\n\rBit" +
- "BatchChunk\">\n\tBitStatus\0221\n\017fragment_stat" +
- "us\030\001 \003(\0132\030.exec.bit.FragmentStatus\"\251\003\n\016F" +
- "ragmentStatus\022\022\n\nmemory_use\030\001 \001(\003\022\031\n\021bat" +
- "ches_completed\030\002 \001(\003\022\031\n\021records_complete" +
- "d\030\003 \001(\003\022\'\n\037estimated_completion_percenta" +
- "ge\030\004 \001(\005\0225\n\005state\030\005 \001(\0162&.exec.bit.Fragm",
- "entStatus.FragmentState\022\026\n\016data_processe" +
- "d\030\006 \001(\003\022(\n\006handle\030\007 \001(\0132\030.exec.bit.Fragm" +
- "entHandle\022(\n\005error\030\010 \001(\0132\031.exec.shared.D" +
- "rillPBError\022\024\n\014running_time\030\t \001(\003\"k\n\rFra" +
- "gmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALLO" +
- "CATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\tC" +
- "ANCELLED\020\004\022\n\n\006FAILED\020\005\"\304\001\n\023FragmentRecor" +
- "dBatch\022(\n\006handle\030\001 \001(\0132\030.exec.bit.Fragme" +
- "ntHandle\022!\n\031sending_major_fragment_id\030\002 " +
- "\001(\005\022!\n\031sending_minor_fragment_id\030\003 \001(\005\022(",
- "\n\003def\030\004 \001(\0132\033.exec.shared.RecordBatchDef" +
- "\022\023\n\013isLastBatch\030\005 \001(\010\"\213\002\n\014PlanFragment\022(" +
- "\n\006handle\030\001 \001(\0132\030.exec.bit.FragmentHandle" +
- "\022\024\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_cost\030\005 \001(\002" +
- "\022\021\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_cost\030\007 \001(\002" +
- "\022\025\n\rfragment_json\030\010 \001(\t\022*\n\nassignment\030\n " +
- "\001(\0132\026.exec.DrillbitEndpoint\022\025\n\rleaf_frag" +
- "ment\030\t \001(\010\022\'\n\007foreman\030\013 \001(\0132\026.exec.Drill" +
- "bitEndpoint\"n\n\016FragmentHandle\022&\n\010query_i" +
- "d\030\001 \001(\0132\024.exec.shared.QueryId\022\031\n\021major_f",
- "ragment_id\030\002 \001(\005\022\031\n\021minor_fragment_id\030\003 " +
- "\001(\005\"f\n\017WorkQueueStatus\022(\n\010endpoint\030\001 \001(\013" +
- "2\026.exec.DrillbitEndpoint\022\024\n\014queue_length" +
- "\030\002 \001(\005\022\023\n\013report_time\030\003 \001(\003*\360\001\n\007RpcType\022" +
- "\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\033\n\027" +
- "REQ_INIATILIZE_FRAGMENT\020\003\022\024\n\020REQ_RECORD_" +
- "BATCH\020\004\022\027\n\023REQ_CANCEL_FRAGMENT\020\006\022\027\n\023REQ_" +
- "FRAGMENT_STATUS\020\007\022\022\n\016REQ_BIT_STATUS\020\010\022\030\n" +
- "\024RESP_FRAGMENT_HANDLE\020\t\022\030\n\024RESP_FRAGMENT" +
- "_STATUS\020\n\022\023\n\017RESP_BIT_STATUS\020\013B+\n\033org.ap",
- "ache.drill.exec.protoB\nExecProtosH\001"
+ "ination.proto\032\023UserBitShared.proto\"n\n\016Fr" +
+ "agmentHandle\022&\n\010query_id\030\001 \001(\0132\024.exec.sh" +
+ "ared.QueryId\022\031\n\021major_fragment_id\030\002 \001(\005\022" +
+ "\031\n\021minor_fragment_id\030\003 \001(\005B+\n\033org.apache" +
+ ".drill.exec.protoB\nExecProtosH\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
public com.google.protobuf.ExtensionRegistry assignDescriptors(
com.google.protobuf.Descriptors.FileDescriptor root) {
descriptor = root;
- internal_static_exec_bit_BitHandshake_descriptor =
- getDescriptor().getMessageTypes().get(0);
- internal_static_exec_bit_BitHandshake_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_exec_bit_BitHandshake_descriptor,
- new java.lang.String[] { "RpcVersion", "Endpoint", });
- internal_static_exec_bit_BitBatchChunk_descriptor =
- getDescriptor().getMessageTypes().get(1);
- internal_static_exec_bit_BitBatchChunk_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_exec_bit_BitBatchChunk_descriptor,
- new java.lang.String[] { });
- internal_static_exec_bit_BitStatus_descriptor =
- getDescriptor().getMessageTypes().get(2);
- internal_static_exec_bit_BitStatus_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_exec_bit_BitStatus_descriptor,
- new java.lang.String[] { "FragmentStatus", });
- internal_static_exec_bit_FragmentStatus_descriptor =
- getDescriptor().getMessageTypes().get(3);
- internal_static_exec_bit_FragmentStatus_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_exec_bit_FragmentStatus_descriptor,
- new java.lang.String[] { "MemoryUse", "BatchesCompleted", "RecordsCompleted", "EstimatedCompletionPercentage", "State", "DataProcessed", "Handle", "Error", "RunningTime", });
- internal_static_exec_bit_FragmentRecordBatch_descriptor =
- getDescriptor().getMessageTypes().get(4);
- internal_static_exec_bit_FragmentRecordBatch_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_exec_bit_FragmentRecordBatch_descriptor,
- new java.lang.String[] { "Handle", "SendingMajorFragmentId", "SendingMinorFragmentId", "Def", "IsLastBatch", });
- internal_static_exec_bit_PlanFragment_descriptor =
- getDescriptor().getMessageTypes().get(5);
- internal_static_exec_bit_PlanFragment_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_exec_bit_PlanFragment_descriptor,
- new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "Assignment", "LeafFragment", "Foreman", });
internal_static_exec_bit_FragmentHandle_descriptor =
- getDescriptor().getMessageTypes().get(6);
+ getDescriptor().getMessageTypes().get(0);
internal_static_exec_bit_FragmentHandle_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_exec_bit_FragmentHandle_descriptor,
new java.lang.String[] { "QueryId", "MajorFragmentId", "MinorFragmentId", });
- internal_static_exec_bit_WorkQueueStatus_descriptor =
- getDescriptor().getMessageTypes().get(7);
- internal_static_exec_bit_WorkQueueStatus_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_exec_bit_WorkQueueStatus_descriptor,
- new java.lang.String[] { "Endpoint", "QueueLength", "ReportTime", });
return null;
}
};
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index 60dd6fd9f..1b6d24275 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -25,6 +25,97 @@ public final class UserBitShared {
public static void registerAllExtensions(
com.google.protobuf.ExtensionRegistry registry) {
}
+ /**
+ * Protobuf enum {@code exec.shared.RpcChannel}
+ */
+ public enum RpcChannel
+ implements com.google.protobuf.ProtocolMessageEnum {
+ /**
+ * <code>BIT_CONTROL = 0;</code>
+ */
+ BIT_CONTROL(0, 0),
+ /**
+ * <code>BIT_DATA = 1;</code>
+ */
+ BIT_DATA(1, 1),
+ /**
+ * <code>USER = 2;</code>
+ */
+ USER(2, 2),
+ ;
+
+ /**
+ * <code>BIT_CONTROL = 0;</code>
+ */
+ public static final int BIT_CONTROL_VALUE = 0;
+ /**
+ * <code>BIT_DATA = 1;</code>
+ */
+ public static final int BIT_DATA_VALUE = 1;
+ /**
+ * <code>USER = 2;</code>
+ */
+ public static final int USER_VALUE = 2;
+
+
+ public final int getNumber() { return value; }
+
+ public static RpcChannel valueOf(int value) {
+ switch (value) {
+ case 0: return BIT_CONTROL;
+ case 1: return BIT_DATA;
+ case 2: return USER;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap<RpcChannel>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap<RpcChannel>
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap<RpcChannel>() {
+ public RpcChannel findValueByNumber(int number) {
+ return RpcChannel.valueOf(number);
+ }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(index);
+ }
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return org.apache.drill.exec.proto.UserBitShared.getDescriptor().getEnumTypes().get(0);
+ }
+
+ private static final RpcChannel[] VALUES = values();
+
+ public static RpcChannel valueOf(
+ com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+
+ private final int index;
+ private final int value;
+
+ private RpcChannel(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:exec.shared.RpcChannel)
+ }
+
public interface QueryIdOrBuilder
extends com.google.protobuf.MessageOrBuilder {
@@ -2563,315 +2654,6 @@ public final class UserBitShared {
// @@protoc_insertion_point(class_scope:exec.shared.ParsingError)
}
- public interface RecordBatchOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
- }
- /**
- * Protobuf type {@code exec.shared.RecordBatch}
- */
- public static final class RecordBatch extends
- com.google.protobuf.GeneratedMessage
- implements RecordBatchOrBuilder {
- // Use RecordBatch.newBuilder() to construct.
- private RecordBatch(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
- super(builder);
- this.unknownFields = builder.getUnknownFields();
- }
- private RecordBatch(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
- private static final RecordBatch defaultInstance;
- public static RecordBatch getDefaultInstance() {
- return defaultInstance;
- }
-
- public RecordBatch getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- private final com.google.protobuf.UnknownFieldSet unknownFields;
- @java.lang.Override
- public final com.google.protobuf.UnknownFieldSet
- getUnknownFields() {
- return this.unknownFields;
- }
- private RecordBatch(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- initFields();
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder();
- try {
- boolean done = false;
- while (!done) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- done = true;
- break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- done = true;
- }
- break;
- }
- }
- }
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- throw e.setUnfinishedMessage(this);
- } catch (java.io.IOException e) {
- throw new com.google.protobuf.InvalidProtocolBufferException(
- e.getMessage()).setUnfinishedMessage(this);
- } finally {
- this.unknownFields = unknownFields.build();
- makeExtensionsImmutable();
- }
- }
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_RecordBatch_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_RecordBatch_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.UserBitShared.RecordBatch.class, org.apache.drill.exec.proto.UserBitShared.RecordBatch.Builder.class);
- }
-
- public static com.google.protobuf.Parser<RecordBatch> PARSER =
- new com.google.protobuf.AbstractParser<RecordBatch>() {
- public RecordBatch parsePartialFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return new RecordBatch(input, extensionRegistry);
- }
- };
-
- @java.lang.Override
- public com.google.protobuf.Parser<RecordBatch> getParserForType() {
- return PARSER;
- }
-
- private void initFields() {
- }
- private byte memoizedIsInitialized = -1;
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) return isInitialized == 1;
-
- memoizedIsInitialized = 1;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @java.lang.Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
- }
-
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input);
- }
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input, extensionRegistry);
- }
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.drill.exec.proto.UserBitShared.RecordBatch parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.drill.exec.proto.UserBitShared.RecordBatch prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- @java.lang.Override
- protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
- }
- /**
- * Protobuf type {@code exec.shared.RecordBatch}
- */
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.drill.exec.proto.UserBitShared.RecordBatchOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_RecordBatch_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_RecordBatch_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.drill.exec.proto.UserBitShared.RecordBatch.class, org.apache.drill.exec.proto.UserBitShared.RecordBatch.Builder.class);
- }
-
- // Construct using org.apache.drill.exec.proto.UserBitShared.RecordBatch.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- }
- }
- private static Builder create() {
- return new Builder();
- }
-
- public Builder clear() {
- super.clear();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(buildPartial());
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_RecordBatch_descriptor;
- }
-
- public org.apache.drill.exec.proto.UserBitShared.RecordBatch getDefaultInstanceForType() {
- return org.apache.drill.exec.proto.UserBitShared.RecordBatch.getDefaultInstance();
- }
-
- public org.apache.drill.exec.proto.UserBitShared.RecordBatch build() {
- org.apache.drill.exec.proto.UserBitShared.RecordBatch result = buildPartial();
- if (!result.isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return result;
- }
-
- public org.apache.drill.exec.proto.UserBitShared.RecordBatch buildPartial() {
- org.apache.drill.exec.proto.UserBitShared.RecordBatch result = new org.apache.drill.exec.proto.UserBitShared.RecordBatch(this);
- onBuilt();
- return result;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.drill.exec.proto.UserBitShared.RecordBatch) {
- return mergeFrom((org.apache.drill.exec.proto.UserBitShared.RecordBatch)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(org.apache.drill.exec.proto.UserBitShared.RecordBatch other) {
- if (other == org.apache.drill.exec.proto.UserBitShared.RecordBatch.getDefaultInstance()) return this;
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public final boolean isInitialized() {
- return true;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- org.apache.drill.exec.proto.UserBitShared.RecordBatch parsedMessage = null;
- try {
- parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.drill.exec.proto.UserBitShared.RecordBatch) e.getUnfinishedMessage();
- throw e;
- } finally {
- if (parsedMessage != null) {
- mergeFrom(parsedMessage);
- }
- }
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:exec.shared.RecordBatch)
- }
-
- static {
- defaultInstance = new RecordBatch(true);
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:exec.shared.RecordBatch)
- }
-
public interface RecordBatchDefOrBuilder
extends com.google.protobuf.MessageOrBuilder {
@@ -4952,11 +4734,6 @@ public final class UserBitShared {
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_exec_shared_ParsingError_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
- internal_static_exec_shared_RecordBatch_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_exec_shared_RecordBatch_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
internal_static_exec_shared_RecordBatchDef_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -4984,14 +4761,15 @@ public final class UserBitShared {
"ror\030\005 \003(\0132\031.exec.shared.ParsingError\"\\\n\014" +
"ParsingError\022\024\n\014start_column\030\002 \001(\005\022\021\n\tst" +
"art_row\030\003 \001(\005\022\022\n\nend_column\030\004 \001(\005\022\017\n\007end" +
- "_row\030\005 \001(\005\"\r\n\013RecordBatch\"p\n\016RecordBatch",
- "Def\022)\n\005field\030\001 \003(\0132\032.exec.shared.FieldMe" +
- "tadata\022\024\n\014record_count\030\002 \001(\005\022\035\n\025is_selec" +
- "tion_vector_2\030\003 \001(\010\"\261\001\n\rFieldMetadata\022\033\n" +
- "\003def\030\001 \001(\0132\016.exec.FieldDef\022\023\n\013value_coun" +
- "t\030\002 \001(\005\022\027\n\017var_byte_length\030\003 \001(\005\022\023\n\013grou" +
- "p_count\030\004 \001(\005\022\025\n\rbuffer_length\030\005 \001(\005\022)\n\005" +
- "child\030\006 \003(\0132\032.exec.shared.FieldMetadataB" +
+ "_row\030\005 \001(\005\"p\n\016RecordBatchDef\022)\n\005field\030\001 ",
+ "\003(\0132\032.exec.shared.FieldMetadata\022\024\n\014recor" +
+ "d_count\030\002 \001(\005\022\035\n\025is_selection_vector_2\030\003" +
+ " \001(\010\"\261\001\n\rFieldMetadata\022\033\n\003def\030\001 \001(\0132\016.ex" +
+ "ec.FieldDef\022\023\n\013value_count\030\002 \001(\005\022\027\n\017var_" +
+ "byte_length\030\003 \001(\005\022\023\n\013group_count\030\004 \001(\005\022\025" +
+ "\n\rbuffer_length\030\005 \001(\005\022)\n\005child\030\006 \003(\0132\032.e" +
+ "xec.shared.FieldMetadata*5\n\nRpcChannel\022\017" +
+ "\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002B" +
".\n\033org.apache.drill.exec.protoB\rUserBitS" +
"haredH\001"
};
@@ -5018,20 +4796,14 @@ public final class UserBitShared {
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_exec_shared_ParsingError_descriptor,
new java.lang.String[] { "StartColumn", "StartRow", "EndColumn", "EndRow", });
- internal_static_exec_shared_RecordBatch_descriptor =
- getDescriptor().getMessageTypes().get(3);
- internal_static_exec_shared_RecordBatch_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_exec_shared_RecordBatch_descriptor,
- new java.lang.String[] { });
internal_static_exec_shared_RecordBatchDef_descriptor =
- getDescriptor().getMessageTypes().get(4);
+ getDescriptor().getMessageTypes().get(3);
internal_static_exec_shared_RecordBatchDef_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_exec_shared_RecordBatchDef_descriptor,
new java.lang.String[] { "Field", "RecordCount", "IsSelectionVector2", });
internal_static_exec_shared_FieldMetadata_descriptor =
- getDescriptor().getMessageTypes().get(5);
+ getDescriptor().getMessageTypes().get(4);
internal_static_exec_shared_FieldMetadata_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_exec_shared_FieldMetadata_descriptor,
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
index e505204b9..8d4fc602e 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
@@ -356,6 +356,16 @@ public final class UserProtos {
public interface UserToBitHandshakeOrBuilder
extends com.google.protobuf.MessageOrBuilder {
+ // optional .exec.shared.RpcChannel channel = 1 [default = USER];
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 1 [default = USER];</code>
+ */
+ boolean hasChannel();
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 1 [default = USER];</code>
+ */
+ org.apache.drill.exec.proto.UserBitShared.RpcChannel getChannel();
+
// optional bool support_listening = 2;
/**
* <code>optional bool support_listening = 2;</code>
@@ -427,13 +437,24 @@ public final class UserProtos {
}
break;
}
+ case 8: {
+ int rawValue = input.readEnum();
+ org.apache.drill.exec.proto.UserBitShared.RpcChannel value = org.apache.drill.exec.proto.UserBitShared.RpcChannel.valueOf(rawValue);
+ if (value == null) {
+ unknownFields.mergeVarintField(1, rawValue);
+ } else {
+ bitField0_ |= 0x00000001;
+ channel_ = value;
+ }
+ break;
+ }
case 16: {
- bitField0_ |= 0x00000001;
+ bitField0_ |= 0x00000002;
supportListening_ = input.readBool();
break;
}
case 24: {
- bitField0_ |= 0x00000002;
+ bitField0_ |= 0x00000004;
rpcVersion_ = input.readInt32();
break;
}
@@ -477,6 +498,22 @@ public final class UserProtos {
}
private int bitField0_;
+ // optional .exec.shared.RpcChannel channel = 1 [default = USER];
+ public static final int CHANNEL_FIELD_NUMBER = 1;
+ private org.apache.drill.exec.proto.UserBitShared.RpcChannel channel_;
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 1 [default = USER];</code>
+ */
+ public boolean hasChannel() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 1 [default = USER];</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RpcChannel getChannel() {
+ return channel_;
+ }
+
// optional bool support_listening = 2;
public static final int SUPPORT_LISTENING_FIELD_NUMBER = 2;
private boolean supportListening_;
@@ -484,7 +521,7 @@ public final class UserProtos {
* <code>optional bool support_listening = 2;</code>
*/
public boolean hasSupportListening() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
+ return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
* <code>optional bool support_listening = 2;</code>
@@ -500,7 +537,7 @@ public final class UserProtos {
* <code>optional int32 rpc_version = 3;</code>
*/
public boolean hasRpcVersion() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
+ return ((bitField0_ & 0x00000004) == 0x00000004);
}
/**
* <code>optional int32 rpc_version = 3;</code>
@@ -510,6 +547,7 @@ public final class UserProtos {
}
private void initFields() {
+ channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.USER;
supportListening_ = false;
rpcVersion_ = 0;
}
@@ -526,9 +564,12 @@ public final class UserProtos {
throws java.io.IOException {
getSerializedSize();
if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeBool(2, supportListening_);
+ output.writeEnum(1, channel_.getNumber());
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeBool(2, supportListening_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
output.writeInt32(3, rpcVersion_);
}
getUnknownFields().writeTo(output);
@@ -542,10 +583,14 @@ public final class UserProtos {
size = 0;
if (((bitField0_ & 0x00000001) == 0x00000001)) {
size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(2, supportListening_);
+ .computeEnumSize(1, channel_.getNumber());
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(2, supportListening_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
.computeInt32Size(3, rpcVersion_);
}
size += getUnknownFields().getSerializedSize();
@@ -664,10 +709,12 @@ public final class UserProtos {
public Builder clear() {
super.clear();
- supportListening_ = false;
+ channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.USER;
bitField0_ = (bitField0_ & ~0x00000001);
- rpcVersion_ = 0;
+ supportListening_ = false;
bitField0_ = (bitField0_ & ~0x00000002);
+ rpcVersion_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000004);
return this;
}
@@ -699,10 +746,14 @@ public final class UserProtos {
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
to_bitField0_ |= 0x00000001;
}
- result.supportListening_ = supportListening_;
+ result.channel_ = channel_;
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
to_bitField0_ |= 0x00000002;
}
+ result.supportListening_ = supportListening_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
result.rpcVersion_ = rpcVersion_;
result.bitField0_ = to_bitField0_;
onBuilt();
@@ -720,6 +771,9 @@ public final class UserProtos {
public Builder mergeFrom(org.apache.drill.exec.proto.UserProtos.UserToBitHandshake other) {
if (other == org.apache.drill.exec.proto.UserProtos.UserToBitHandshake.getDefaultInstance()) return this;
+ if (other.hasChannel()) {
+ setChannel(other.getChannel());
+ }
if (other.hasSupportListening()) {
setSupportListening(other.getSupportListening());
}
@@ -753,13 +807,49 @@ public final class UserProtos {
}
private int bitField0_;
+ // optional .exec.shared.RpcChannel channel = 1 [default = USER];
+ private org.apache.drill.exec.proto.UserBitShared.RpcChannel channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.USER;
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 1 [default = USER];</code>
+ */
+ public boolean hasChannel() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 1 [default = USER];</code>
+ */
+ public org.apache.drill.exec.proto.UserBitShared.RpcChannel getChannel() {
+ return channel_;
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 1 [default = USER];</code>
+ */
+ public Builder setChannel(org.apache.drill.exec.proto.UserBitShared.RpcChannel value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ channel_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional .exec.shared.RpcChannel channel = 1 [default = USER];</code>
+ */
+ public Builder clearChannel() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.USER;
+ onChanged();
+ return this;
+ }
+
// optional bool support_listening = 2;
private boolean supportListening_ ;
/**
* <code>optional bool support_listening = 2;</code>
*/
public boolean hasSupportListening() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
+ return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
* <code>optional bool support_listening = 2;</code>
@@ -771,7 +861,7 @@ public final class UserProtos {
* <code>optional bool support_listening = 2;</code>
*/
public Builder setSupportListening(boolean value) {
- bitField0_ |= 0x00000001;
+ bitField0_ |= 0x00000002;
supportListening_ = value;
onChanged();
return this;
@@ -780,7 +870,7 @@ public final class UserProtos {
* <code>optional bool support_listening = 2;</code>
*/
public Builder clearSupportListening() {
- bitField0_ = (bitField0_ & ~0x00000001);
+ bitField0_ = (bitField0_ & ~0x00000002);
supportListening_ = false;
onChanged();
return this;
@@ -792,7 +882,7 @@ public final class UserProtos {
* <code>optional int32 rpc_version = 3;</code>
*/
public boolean hasRpcVersion() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
+ return ((bitField0_ & 0x00000004) == 0x00000004);
}
/**
* <code>optional int32 rpc_version = 3;</code>
@@ -804,7 +894,7 @@ public final class UserProtos {
* <code>optional int32 rpc_version = 3;</code>
*/
public Builder setRpcVersion(int value) {
- bitField0_ |= 0x00000002;
+ bitField0_ |= 0x00000004;
rpcVersion_ = value;
onChanged();
return this;
@@ -813,7 +903,7 @@ public final class UserProtos {
* <code>optional int32 rpc_version = 3;</code>
*/
public Builder clearRpcVersion() {
- bitField0_ = (bitField0_ & ~0x00000002);
+ bitField0_ = (bitField0_ & ~0x00000004);
rpcVersion_ = 0;
onChanged();
return this;
@@ -5099,35 +5189,36 @@ public final class UserProtos {
static {
java.lang.String[] descriptorData = {
"\n\nUser.proto\022\texec.user\032\017SchemaDef.proto" +
- "\032\023UserBitShared.proto\"D\n\022UserToBitHandsh" +
- "ake\022\031\n\021support_listening\030\002 \001(\010\022\023\n\013rpc_ve" +
- "rsion\030\003 \001(\005\"S\n\016RequestResults\022&\n\010query_i" +
- "d\030\001 \001(\0132\024.exec.shared.QueryId\022\031\n\021maximum" +
- "_responses\030\002 \001(\005\"o\n\010RunQuery\0221\n\014results_" +
- "mode\030\001 \001(\0162\033.exec.user.QueryResultsMode\022" +
- "\"\n\004type\030\002 \001(\0162\024.exec.user.QueryType\022\014\n\004p" +
- "lan\030\003 \001(\t\")\n\022BitToUserHandshake\022\023\n\013rpc_v" +
- "ersion\030\002 \001(\005\"7\n\nNodeStatus\022\017\n\007node_id\030\001 ",
- "\001(\005\022\030\n\020memory_footprint\030\002 \001(\003\"\331\003\n\013QueryR" +
- "esult\0226\n\013query_state\030\001 \001(\0162!.exec.user.Q" +
- "ueryResult.QueryState\022&\n\010query_id\030\002 \001(\0132" +
- "\024.exec.shared.QueryId\022\025\n\ris_last_chunk\030\003" +
- " \001(\010\022\021\n\trow_count\030\004 \001(\005\022\024\n\014records_scan\030" +
- "\005 \001(\003\022\025\n\rrecords_error\030\006 \001(\003\022\027\n\017submissi" +
- "on_time\030\007 \001(\003\022*\n\013node_status\030\010 \003(\0132\025.exe" +
- "c.user.NodeStatus\022(\n\005error\030\t \003(\0132\031.exec." +
- "shared.DrillPBError\022(\n\003def\030\n \001(\0132\033.exec." +
- "shared.RecordBatchDef\022\026\n\016schema_changed\030",
- "\013 \001(\010\"b\n\nQueryState\022\013\n\007PENDING\020\000\022\013\n\007RUNN" +
- "ING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n\010CANCELED\020\003\022\n\n\006FA" +
- "ILED\020\004\022\021\n\rUNKNOWN_QUERY\020\005*\210\001\n\007RpcType\022\r\n" +
- "\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\r\n\tRU" +
- "N_QUERY\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017REQUEST_R" +
- "ESULTS\020\005\022\020\n\014QUERY_RESULT\020\006\022\020\n\014QUERY_HAND" +
- "LE\020\007*/\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022" +
- "\014\n\010PHYSICAL\020\003*#\n\020QueryResultsMode\022\017\n\013STR" +
- "EAM_FULL\020\001B+\n\033org.apache.drill.exec.prot" +
- "oB\nUserProtosH\001"
+ "\032\023UserBitShared.proto\"t\n\022UserToBitHandsh" +
+ "ake\022.\n\007channel\030\001 \001(\0162\027.exec.shared.RpcCh" +
+ "annel:\004USER\022\031\n\021support_listening\030\002 \001(\010\022\023" +
+ "\n\013rpc_version\030\003 \001(\005\"S\n\016RequestResults\022&\n" +
+ "\010query_id\030\001 \001(\0132\024.exec.shared.QueryId\022\031\n" +
+ "\021maximum_responses\030\002 \001(\005\"o\n\010RunQuery\0221\n\014" +
+ "results_mode\030\001 \001(\0162\033.exec.user.QueryResu" +
+ "ltsMode\022\"\n\004type\030\002 \001(\0162\024.exec.user.QueryT" +
+ "ype\022\014\n\004plan\030\003 \001(\t\")\n\022BitToUserHandshake\022",
+ "\023\n\013rpc_version\030\002 \001(\005\"7\n\nNodeStatus\022\017\n\007no" +
+ "de_id\030\001 \001(\005\022\030\n\020memory_footprint\030\002 \001(\003\"\331\003" +
+ "\n\013QueryResult\0226\n\013query_state\030\001 \001(\0162!.exe" +
+ "c.user.QueryResult.QueryState\022&\n\010query_i" +
+ "d\030\002 \001(\0132\024.exec.shared.QueryId\022\025\n\ris_last" +
+ "_chunk\030\003 \001(\010\022\021\n\trow_count\030\004 \001(\005\022\024\n\014recor" +
+ "ds_scan\030\005 \001(\003\022\025\n\rrecords_error\030\006 \001(\003\022\027\n\017" +
+ "submission_time\030\007 \001(\003\022*\n\013node_status\030\010 \003" +
+ "(\0132\025.exec.user.NodeStatus\022(\n\005error\030\t \003(\013" +
+ "2\031.exec.shared.DrillPBError\022(\n\003def\030\n \001(\013",
+ "2\033.exec.shared.RecordBatchDef\022\026\n\016schema_" +
+ "changed\030\013 \001(\010\"b\n\nQueryState\022\013\n\007PENDING\020\000" +
+ "\022\013\n\007RUNNING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n\010CANCELED" +
+ "\020\003\022\n\n\006FAILED\020\004\022\021\n\rUNKNOWN_QUERY\020\005*\210\001\n\007Rp" +
+ "cType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE" +
+ "\020\002\022\r\n\tRUN_QUERY\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017R" +
+ "EQUEST_RESULTS\020\005\022\020\n\014QUERY_RESULT\020\006\022\020\n\014QU" +
+ "ERY_HANDLE\020\007*/\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LO" +
+ "GICAL\020\002\022\014\n\010PHYSICAL\020\003*#\n\020QueryResultsMod" +
+ "e\022\017\n\013STREAM_FULL\020\001B+\n\033org.apache.drill.e",
+ "xec.protoB\nUserProtosH\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -5139,7 +5230,7 @@ public final class UserProtos {
internal_static_exec_user_UserToBitHandshake_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_exec_user_UserToBitHandshake_descriptor,
- new java.lang.String[] { "SupportListening", "RpcVersion", });
+ new java.lang.String[] { "Channel", "SupportListening", "RpcVersion", });
internal_static_exec_user_RequestResults_descriptor =
getDescriptor().getMessageTypes().get(1);
internal_static_exec_user_RequestResults_fieldAccessorTable = new
diff --git a/protocol/src/main/protobuf/BitControl.proto b/protocol/src/main/protobuf/BitControl.proto
new file mode 100644
index 000000000..d96f7cf0d
--- /dev/null
+++ b/protocol/src/main/protobuf/BitControl.proto
@@ -0,0 +1,83 @@
+package exec.bit.control;
+
+option java_package = "org.apache.drill.exec.proto";
+option java_outer_classname = "BitControl";
+option optimize_for = SPEED;
+
+import "ExecutionProtos.proto";
+import "Coordination.proto";
+import "UserBitShared.proto";
+
+
+////// BitControl RPC ///////
+enum RpcType {
+ HANDSHAKE = 0;
+ ACK = 1;
+ GOODBYE = 2;
+
+ // bit requests
+ REQ_INIATILIZE_FRAGMENT = 3; // Returns Handle
+
+ REQ_CANCEL_FRAGMENT = 6; // send a cancellation message for a fragment, returns Ack
+ REQ_FRAGMENT_STATUS = 7; // get a fragment status, returns FragmentStatus
+ REQ_BIT_STATUS = 8; // get bit status.
+
+ // bit responses
+ RESP_FRAGMENT_HANDLE = 9;
+ RESP_FRAGMENT_STATUS = 10;
+ RESP_BIT_STATUS = 11;
+}
+
+
+message BitControlHandshake{
+ optional int32 rpc_version = 1;
+ optional exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];
+ optional DrillbitEndpoint endpoint = 3;
+}
+
+message BitStatus {
+ repeated FragmentStatus fragment_status = 1;
+}
+
+message FragmentStatus {
+
+ enum FragmentState {
+ SENDING = 0;
+ AWAITING_ALLOCATION = 1;
+ RUNNING = 2;
+ FINISHED = 3;
+ CANCELLED = 4;
+ FAILED = 5;
+ }
+
+ optional int64 memory_use = 1;
+ optional int64 batches_completed = 2;
+ optional int64 records_completed = 3;
+ optional int32 estimated_completion_percentage = 4;
+ optional FragmentState state = 5;
+ optional int64 data_processed = 6;
+
+ optional FragmentHandle handle = 7;
+ optional exec.shared.DrillPBError error = 8;
+ optional int64 running_time = 9;
+}
+
+message PlanFragment {
+ optional FragmentHandle handle = 1;
+ optional float network_cost = 4;
+ optional float cpu_cost = 5;
+ optional float disk_cost = 6;
+ optional float memory_cost = 7;
+ optional string fragment_json = 8;
+ optional DrillbitEndpoint assignment = 10;
+ optional bool leaf_fragment = 9;
+ optional DrillbitEndpoint foreman = 11;
+ optional int64 mem_initial = 12 [default = 20000000]; // 20 megs
+ optional int64 mem_max = 13 [default = 20000000000]; // 20 gigs
+}
+
+message WorkQueueStatus{
+ optional DrillbitEndpoint endpoint = 1;
+ optional int32 queue_length = 2;
+ optional int64 report_time = 3;
+}
diff --git a/protocol/src/main/protobuf/BitData.proto b/protocol/src/main/protobuf/BitData.proto
new file mode 100644
index 000000000..53563989d
--- /dev/null
+++ b/protocol/src/main/protobuf/BitData.proto
@@ -0,0 +1,34 @@
+package exec.bit.data;
+
+option java_package = "org.apache.drill.exec.proto";
+option java_outer_classname = "BitData";
+option optimize_for = SPEED;
+
+import "ExecutionProtos.proto";
+import "Coordination.proto";
+import "UserBitShared.proto";
+
+enum RpcType {
+ HANDSHAKE = 0;
+ ACK = 1;
+ GOODBYE = 2;
+ REQ_RECORD_BATCH = 3; // send record batch, returns Ack
+}
+
+message BitClientHandshake{
+ optional int32 rpc_version = 1;
+ optional exec.shared.RpcChannel channel = 2 [default = BIT_DATA];
+ optional FragmentHandle handle = 3;
+}
+
+message BitServerHandshake{
+ optional int32 rpc_version = 1;
+}
+
+message FragmentRecordBatch{
+ optional FragmentHandle handle = 1;
+ optional int32 sending_major_fragment_id = 2;
+ optional int32 sending_minor_fragment_id = 3;
+ optional exec.shared.RecordBatchDef def = 4;
+ optional bool isLastBatch = 5;
+}
diff --git a/protocol/src/main/protobuf/Coordination.proto b/protocol/src/main/protobuf/Coordination.proto
index 5cc5caba1..28c99d6ea 100644
--- a/protocol/src/main/protobuf/Coordination.proto
+++ b/protocol/src/main/protobuf/Coordination.proto
@@ -7,8 +7,9 @@ option optimize_for = SPEED;
message DrillbitEndpoint{
optional string address = 1;
optional int32 user_port = 2;
- optional int32 bit_port = 3;
- optional Roles roles = 4;
+ optional int32 control_port = 3;
+ optional int32 data_port = 4;
+ optional Roles roles = 5;
}
message DrillServiceInstance{
diff --git a/protocol/src/main/protobuf/ExecutionProtos.proto b/protocol/src/main/protobuf/ExecutionProtos.proto
index 7501d7c0e..8ad8c474e 100644
--- a/protocol/src/main/protobuf/ExecutionProtos.proto
+++ b/protocol/src/main/protobuf/ExecutionProtos.proto
@@ -7,92 +7,9 @@ option optimize_for = SPEED;
import "Coordination.proto";
import "UserBitShared.proto";
-
-
-////// UserToBit RPC ///////
-enum RpcType {
- HANDSHAKE = 0;
- ACK = 1;
- GOODBYE = 2;
-
- // bit requests
- REQ_INIATILIZE_FRAGMENT = 3; // Returns Handle
- REQ_RECORD_BATCH = 4; // send record batch overview, returns Ack
-
- REQ_CANCEL_FRAGMENT = 6; // send a cancellation message for a fragment, returns Ack
- REQ_FRAGMENT_STATUS = 7; // get a fragment status, returns FragmentStatus
- REQ_BIT_STATUS = 8; // get bit status.
-
- // bit responses
- RESP_FRAGMENT_HANDLE = 9;
- RESP_FRAGMENT_STATUS = 10;
- RESP_BIT_STATUS = 11;
-}
-
-
-message BitHandshake{
- optional int32 rpc_version = 1;
- optional DrillbitEndpoint endpoint = 2;
-}
-
-message BitBatchChunk {}
-
-message BitStatus {
- repeated FragmentStatus fragment_status = 1;
-}
-
-message FragmentStatus {
-
- enum FragmentState {
- SENDING = 0;
- AWAITING_ALLOCATION = 1;
- RUNNING = 2;
- FINISHED = 3;
- CANCELLED = 4;
- FAILED = 5;
- }
-
- optional int64 memory_use = 1;
- optional int64 batches_completed = 2;
- optional int64 records_completed = 3;
- optional int32 estimated_completion_percentage = 4;
- optional FragmentState state = 5;
- optional int64 data_processed = 6;
-
- optional FragmentHandle handle = 7;
- optional exec.shared.DrillPBError error = 8;
- optional int64 running_time = 9;
-}
-
-message FragmentRecordBatch{
- optional FragmentHandle handle = 1;
- optional int32 sending_major_fragment_id = 2;
- optional int32 sending_minor_fragment_id = 3;
- optional exec.shared.RecordBatchDef def = 4;
- optional bool isLastBatch = 5;
-}
-
-message PlanFragment {
- optional FragmentHandle handle = 1;
- optional float network_cost = 4;
- optional float cpu_cost = 5;
- optional float disk_cost = 6;
- optional float memory_cost = 7;
- optional string fragment_json = 8;
- optional DrillbitEndpoint assignment = 10;
- optional bool leaf_fragment = 9;
- optional DrillbitEndpoint foreman = 11;
-
-}
-
message FragmentHandle {
optional exec.shared.QueryId query_id = 1;
optional int32 major_fragment_id = 2;
optional int32 minor_fragment_id = 3;
}
-message WorkQueueStatus{
- optional DrillbitEndpoint endpoint = 1;
- optional int32 queue_length = 2;
- optional int64 report_time = 3;
-}
diff --git a/protocol/src/main/protobuf/User.proto b/protocol/src/main/protobuf/User.proto
index cbf5b4c17..0c1e588e3 100644
--- a/protocol/src/main/protobuf/User.proto
+++ b/protocol/src/main/protobuf/User.proto
@@ -26,8 +26,9 @@ enum RpcType {
}
message UserToBitHandshake {
- optional bool support_listening = 2;
- optional int32 rpc_version = 3;
+ optional exec.shared.RpcChannel channel = 1 [default = USER];
+ optional bool support_listening = 2;
+ optional int32 rpc_version = 3;
}
message RequestResults {
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index 0d9879707..d75a7f900 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -7,35 +7,36 @@ option optimize_for = SPEED;
import "Coordination.proto";
import "SchemaDef.proto";
+enum RpcChannel {
+ BIT_CONTROL = 0;
+ BIT_DATA = 1;
+ USER = 2;
+}
+
message QueryId {
- optional sfixed64 part1 = 1;
- optional sfixed64 part2 = 2;
+ optional sfixed64 part1 = 1;
+ optional sfixed64 part2 = 2;
}
message DrillPBError{
- optional string error_id = 1; // for debug tracing purposes
- optional DrillbitEndpoint endpoint = 2;
- optional int32 error_type = 3;
- optional string message = 4;
- repeated ParsingError parsing_error = 5; //optional, used when providing location of error within a piece of text.
+ optional string error_id = 1; // for debug tracing purposes
+ optional DrillbitEndpoint endpoint = 2;
+ optional int32 error_type = 3;
+ optional string message = 4;
+ repeated ParsingError parsing_error = 5; //optional, used when providing location of error within a piece of text.
}
message ParsingError{
- optional int32 start_column = 2;
- optional int32 start_row = 3;
- optional int32 end_column = 4;
- optional int32 end_row = 5;
-}
-
-message RecordBatch{
-
+ optional int32 start_column = 2;
+ optional int32 start_row = 3;
+ optional int32 end_column = 4;
+ optional int32 end_row = 5;
}
message RecordBatchDef {
repeated FieldMetadata field = 1;
optional int32 record_count = 2;
- optional bool is_selection_vector_2 = 3;
-
+ optional bool is_selection_vector_2 = 3;
}
message FieldMetadata {
diff --git a/sqlparser/src/main/java/org/apache/drill/optiq/DrillOptiq.java b/sqlparser/src/main/java/org/apache/drill/optiq/DrillOptiq.java
index 1245cfe7e..189715181 100644
--- a/sqlparser/src/main/java/org/apache/drill/optiq/DrillOptiq.java
+++ b/sqlparser/src/main/java/org/apache/drill/optiq/DrillOptiq.java
@@ -20,7 +20,7 @@ package org.apache.drill.optiq;
import net.hydromatic.linq4j.Ord;
import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.work.FragmentRunnerListener;
+import org.apache.drill.exec.work.fragment.StatusReporter;
import org.eigenbase.rel.RelNode;
import org.eigenbase.relopt.RelOptPlanner;
import org.eigenbase.reltype.RelDataTypeField;